diff --git a/.gitignore b/.gitignore
index c207409e3cfe087723ec38fa75fca68404bf251c..155e785b01beb809a13c45c40d96f04f4dd6343b 100644
--- a/.gitignore
+++ b/.gitignore
@@ -12,6 +12,7 @@ third_party/libmesos.so
 third_party/libmesos.dylib
 conf/java-opts
 conf/spark-env.sh
+conf/streaming-env.sh
 conf/log4j.properties
 docs/_site
 docs/api
@@ -31,4 +32,7 @@ project/plugins/src_managed/
 logs/
 log/
 spark-tests.log
+streaming-tests.log
 dependency-reduced-pom.xml
+.ensime
+.ensime_lucene
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 5f5834720428974ca1f485838555235db7a13651..be2e3580917d873e802dbc48051d2689591c0120 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.0-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -45,11 +45,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-      <activation>
-        <property>
-          <name>!hadoopVersion</name>
-        </property>
-      </activation>
       <dependencies>
         <dependency>
           <groupId>org.spark-project</groupId>
@@ -77,12 +72,6 @@
     </profile>
     <profile>
       <id>hadoop2</id>
-      <activation>
-        <property>
-          <name>hadoopVersion</name>
-          <value>2</value>
-        </property>
-      </activation>
       <dependencies>
         <dependency>
           <groupId>org.spark-project</groupId>
diff --git a/bagel/src/main/scala/spark/bagel/Bagel.scala b/bagel/src/main/scala/spark/bagel/Bagel.scala
index 996ca2a8771e5c5d30b88ff0b97f0a2e3c85b955..5ecdd7d0045fae68116792eae13da43dbdf1bb39 100644
--- a/bagel/src/main/scala/spark/bagel/Bagel.scala
+++ b/bagel/src/main/scala/spark/bagel/Bagel.scala
@@ -4,21 +4,51 @@ import spark._
 import spark.SparkContext._
 
 import scala.collection.mutable.ArrayBuffer
+import storage.StorageLevel
 
 object Bagel extends Logging {
-  def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
-          C : Manifest, A : Manifest](
+  val DEFAULT_STORAGE_LEVEL = StorageLevel.MEMORY_AND_DISK
+
+  /**
+   * Runs a Bagel program.
+   * @param sc [[spark.SparkContext]] to use for the program.
+   * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the Key will be
+   *                 the vertex id.
+   * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often this will be an
+   *                 empty array, i.e. sc.parallelize(Array[K, Message]()).
+   * @param combiner [[spark.bagel.Combiner]] combines multiple individual messages to a given vertex into one
+   *                message before sending (which often involves network I/O).
+   * @param aggregator [[spark.bagel.Aggregator]] performs a reduce across all vertices after each superstep,
+   *                  and provides the result to each vertex in the next superstep.
+   * @param partitioner [[spark.Partitioner]] partitions values by key
+   * @param numPartitions number of partitions across which to split the graph.
+   *                      Default is the default parallelism of the SparkContext
+   * @param storageLevel [[spark.storage.StorageLevel]] to use for caching of intermediate RDDs in each superstep.
+   *                    Defaults to caching in memory.
+   * @param compute function that takes a Vertex, optional set of (possibly combined) messages to the Vertex,
+   *                optional Aggregator and the current superstep,
+   *                and returns a set of (Vertex, outgoing Messages) pairs
+   * @tparam K key
+   * @tparam V vertex type
+   * @tparam M message type
+   * @tparam C combiner
+   * @tparam A aggregator
+   * @return an RDD of (K, V) pairs representing the graph after completion of the program
+   */
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
+          C: Manifest, A: Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
     messages: RDD[(K, M)],
     combiner: Combiner[M, C],
     aggregator: Option[Aggregator[V, A]],
     partitioner: Partitioner,
-    numSplits: Int
+    numPartitions: Int,
+    storageLevel: StorageLevel = DEFAULT_STORAGE_LEVEL
   )(
     compute: (V, Option[C], Option[A], Int) => (V, Array[M])
   ): RDD[(K, V)] = {
-    val splits = if (numSplits != 0) numSplits else sc.defaultParallelism
+    val splits = if (numPartitions != 0) numPartitions else sc.defaultParallelism
 
     var superstep = 0
     var verts = vertices
@@ -32,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))
+        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))
@@ -50,49 +81,95 @@ object Bagel extends Logging {
     verts
   }
 
-  def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
-          C : Manifest](
+  /** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default storage level */
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
     messages: RDD[(K, M)],
     combiner: Combiner[M, C],
     partitioner: Partitioner,
-    numSplits: Int
+    numPartitions: Int
+  )(
+    compute: (V, Option[C], Int) => (V, Array[M])
+  ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
+
+  /** Runs a Bagel program with no [[spark.bagel.Aggregator]] */
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
+    sc: SparkContext,
+    vertices: RDD[(K, V)],
+    messages: RDD[(K, M)],
+    combiner: Combiner[M, C],
+    partitioner: Partitioner,
+    numPartitions: Int,
+    storageLevel: StorageLevel
   )(
     compute: (V, Option[C], Int) => (V, Array[M])
   ): RDD[(K, V)] = {
     run[K, V, M, C, Nothing](
-      sc, vertices, messages, combiner, None, partitioner, numSplits)(
+      sc, vertices, messages, combiner, None, partitioner, numPartitions, storageLevel)(
       addAggregatorArg[K, V, M, C](compute))
   }
 
-  def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest,
-          C : Manifest](
+  /**
+   * Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]]
+   * and default storage level
+   */
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
+    sc: SparkContext,
+    vertices: RDD[(K, V)],
+    messages: RDD[(K, M)],
+    combiner: Combiner[M, C],
+    numPartitions: Int
+  )(
+    compute: (V, Option[C], Int) => (V, Array[M])
+  ): RDD[(K, V)] = run(sc, vertices, messages, combiner, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
+
+  /** Runs a Bagel program with no [[spark.bagel.Aggregator]] and the default [[spark.HashPartitioner]]*/
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
     messages: RDD[(K, M)],
     combiner: Combiner[M, C],
-    numSplits: Int
+    numPartitions: Int,
+    storageLevel: StorageLevel
   )(
     compute: (V, Option[C], Int) => (V, Array[M])
   ): RDD[(K, V)] = {
-    val part = new HashPartitioner(numSplits)
+    val part = new HashPartitioner(numPartitions)
     run[K, V, M, C, Nothing](
-      sc, vertices, messages, combiner, None, part, numSplits)(
+      sc, vertices, messages, combiner, None, part, numPartitions, storageLevel)(
       addAggregatorArg[K, V, M, C](compute))
   }
 
-  def run[K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
+  /**
+   * Runs a Bagel program with no [[spark.bagel.Aggregator]], default [[spark.HashPartitioner]],
+   * [[spark.bagel.DefaultCombiner]] and the default storage level
+   */
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
     sc: SparkContext,
     vertices: RDD[(K, V)],
     messages: RDD[(K, M)],
-    numSplits: Int
+    numPartitions: Int
   )(
     compute: (V, Option[Array[M]], Int) => (V, Array[M])
-  ): RDD[(K, V)] = {
-    val part = new HashPartitioner(numSplits)
+  ): RDD[(K, V)] = run(sc, vertices, messages, numPartitions, DEFAULT_STORAGE_LEVEL)(compute)
+
+  /**
+   * Runs a Bagel program with no [[spark.bagel.Aggregator]], the default [[spark.HashPartitioner]]
+   * and [[spark.bagel.DefaultCombiner]]
+   */
+  def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest](
+    sc: SparkContext,
+    vertices: RDD[(K, V)],
+    messages: RDD[(K, M)],
+    numPartitions: Int,
+    storageLevel: StorageLevel
+   )(
+    compute: (V, Option[Array[M]], Int) => (V, Array[M])
+   ): RDD[(K, V)] = {
+    val part = new HashPartitioner(numPartitions)
     run[K, V, M, Array[M], Nothing](
-      sc, vertices, messages, new DefaultCombiner(), None, part, numSplits)(
+      sc, vertices, messages, new DefaultCombiner(), None, part, numPartitions, storageLevel)(
       addAggregatorArg[K, V, M, Array[M]](compute))
   }
 
@@ -100,7 +177,7 @@ object Bagel extends Logging {
    * Aggregates the given vertices using the given aggregator, if it
    * is specified.
    */
-  private def agg[K, V <: Vertex, A : Manifest](
+  private def agg[K, V <: Vertex, A: Manifest](
     verts: RDD[(K, V)],
     aggregator: Option[Aggregator[V, A]]
   ): Option[A] = aggregator match {
@@ -116,10 +193,11 @@ object Bagel extends Logging {
    * function. Returns the processed RDD, the number of messages
    * created, and the number of active vertices.
    */
-  private def comp[K : Manifest, V <: Vertex, M <: Message[K], C](
+  private def comp[K: Manifest, V <: Vertex, M <: Message[K], C](
     sc: SparkContext,
     grouped: RDD[(K, (Seq[C], Seq[V]))],
-    compute: (V, Option[C]) => (V, Array[M])
+    compute: (V, Option[C]) => (V, Array[M]),
+    storageLevel: StorageLevel
   ): (RDD[(K, (V, Array[M]))], Int, Int) = {
     var numMsgs = sc.accumulator(0)
     var numActiveVerts = sc.accumulator(0)
@@ -137,7 +215,7 @@ object Bagel extends Logging {
           numActiveVerts += 1
 
         Some((newVert, newMsgs))
-    }.cache
+    }.persist(storageLevel)
 
     // Force evaluation of processed RDD for accurate performance measurements
     processed.foreach(x => {})
@@ -149,9 +227,7 @@ object Bagel extends Logging {
    * Converts a compute function that doesn't take an aggregator to
    * one that does, so it can be passed to Bagel.run.
    */
-  private def addAggregatorArg[
-    K : Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C
-  ](
+  private def addAggregatorArg[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C](
     compute: (V, Option[C], Int) => (V, Array[M])
   ): (V, Option[C], Option[Nothing], Int) => (V, Array[M]) = {
     (vert: V, msgs: Option[C], aggregated: Option[Nothing], superstep: Int) =>
@@ -170,7 +246,8 @@ trait Aggregator[V, A] {
   def mergeAggregators(a: A, b: A): A
 }
 
-class DefaultCombiner[M : Manifest] extends Combiner[M, Array[M]] with Serializable {
+/** Default combiner that simply appends messages together (i.e. performs no aggregation) */
+class DefaultCombiner[M: Manifest] extends Combiner[M, Array[M]] with Serializable {
   def createCombiner(msg: M): Array[M] =
     Array(msg)
   def mergeMsg(combiner: Array[M], msg: M): Array[M] =
diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
index 03843019c010cf54c7c0209967d17503078862b1..bc32663e0fde6e9bfd371178525894840f630a47 100644
--- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
@@ -16,7 +16,7 @@ import scala.xml.{XML,NodeSeq}
 object WikipediaPageRank {
   def main(args: Array[String]) {
     if (args.length < 5) {
-      System.err.println("Usage: WikipediaPageRank <inputFile> <threshold> <numSplits> <host> <usePartitioner>")
+      System.err.println("Usage: WikipediaPageRank <inputFile> <threshold> <numPartitions> <host> <usePartitioner>")
       System.exit(-1)
     }
 
@@ -25,7 +25,7 @@ object WikipediaPageRank {
 
     val inputFile = args(0)
     val threshold = args(1).toDouble
-    val numSplits = args(2).toInt
+    val numPartitions = args(2).toInt
     val host = args(3)
     val usePartitioner = args(4).toBoolean
     val sc = new SparkContext(host, "WikipediaPageRank")
@@ -69,7 +69,7 @@ object WikipediaPageRank {
     val result =
         Bagel.run(
           sc, vertices, messages, combiner = new PRCombiner(),
-          numSplits = numSplits)(
+          numPartitions = numPartitions)(
           utils.computeWithCombiner(numVertices, epsilon))
 
     // Print the result
diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
index 06cc8c748b98293819925d6705f17355512d3ca4..9d9d80d809d0d58ac0d6b52f5c581646cf5323b8 100644
--- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
@@ -88,7 +88,7 @@ object WikipediaPageRankStandalone {
     n: Long,
     partitioner: Partitioner,
     usePartitioner: Boolean,
-    numSplits: Int
+    numPartitions: Int
   ): RDD[(String, Double)] = {
     var ranks = links.mapValues { edges => defaultRank }
     for (i <- 1 to numIterations) {
diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties
index 4c99e450bccccff6e67cbd38e0566553764adb2d..83d05cab2f867a188fe78f1ed7bfaad7fecea3ab 100644
--- a/bagel/src/test/resources/log4j.properties
+++ b/bagel/src/test/resources/log4j.properties
@@ -1,8 +1,8 @@
-# Set everything to be logged to the console
+# Set everything to be logged to the file bagel/target/unit-tests.log 
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false
-log4j.appender.file.file=spark-tests.log
+log4j.appender.file.file=bagel/target/unit-tests.log
 log4j.appender.file.layout=org.apache.log4j.PatternLayout
 log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
 
diff --git a/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala
index ca59f46843798a5286171f0a35cb54923d972518..25db395c22128013d259aae8722d567e3c0ff76f 100644
--- a/bagel/src/test/scala/bagel/BagelSuite.scala
+++ b/bagel/src/test/scala/bagel/BagelSuite.scala
@@ -1,19 +1,18 @@
 package spark.bagel
 
 import org.scalatest.{FunSuite, Assertions, BeforeAndAfter}
-import org.scalatest.prop.Checkers
-import org.scalacheck.Arbitrary._
-import org.scalacheck.Gen
-import org.scalacheck.Prop._
+import org.scalatest.concurrent.Timeouts
+import org.scalatest.time.SpanSugar._
 
 import scala.collection.mutable.ArrayBuffer
 
 import spark._
+import storage.StorageLevel
 
 class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable
 class TestMessage(val targetId: String) extends Message[String] with Serializable
 
-class BagelSuite extends FunSuite with Assertions with BeforeAndAfter {
+class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts {
   
   var sc: SparkContext = _
   
@@ -23,9 +22,9 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter {
       sc = null
     }
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
+    System.clearProperty("spark.driver.port")
   }
-  
+
   test("halting by voting") {
     sc = new SparkContext("local", "test")
     val verts = sc.parallelize(Array("a", "b", "c", "d").map(id => (id, new TestVertex(true, 0))))
@@ -36,8 +35,9 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter {
         (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) =>
           (new TestVertex(superstep < numSupersteps - 1, self.age + 1), Array[TestMessage]())
       }
-    for ((id, vert) <- result.collect)
+    for ((id, vert) <- result.collect) {
       assert(vert.age === numSupersteps)
+    }
   }
 
   test("halting by message silence") {
@@ -57,7 +57,44 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter {
             }
         (new TestVertex(self.active, self.age + 1), msgsOut)
       }
-    for ((id, vert) <- result.collect)
+    for ((id, vert) <- result.collect) {
       assert(vert.age === numSupersteps)
+    }
+  }
+
+  test("large number of iterations") {
+    // This tests whether jobs with a large number of iterations finish in a reasonable time,
+    // because non-memoized recursion in RDD or DAGScheduler used to cause them to hang
+    failAfter(10 seconds) {
+      sc = new SparkContext("local", "test")
+      val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0))))
+      val msgs = sc.parallelize(Array[(String, TestMessage)]())
+      val numSupersteps = 50
+      val result =
+        Bagel.run(sc, verts, msgs, sc.defaultParallelism) {
+          (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) =>
+            (new TestVertex(superstep < numSupersteps - 1, self.age + 1), Array[TestMessage]())
+        }
+      for ((id, vert) <- result.collect) {
+        assert(vert.age === numSupersteps)
+      }
+    }
+  }
+
+  test("using non-default persistence level") {
+    failAfter(10 seconds) {
+      sc = new SparkContext("local", "test")
+      val verts = sc.parallelize((1 to 4).map(id => (id.toString, new TestVertex(true, 0))))
+      val msgs = sc.parallelize(Array[(String, TestMessage)]())
+      val numSupersteps = 50
+      val result =
+        Bagel.run(sc, verts, msgs, sc.defaultParallelism, StorageLevel.DISK_ONLY) {
+          (self: TestVertex, msgs: Option[Array[TestMessage]], superstep: Int) =>
+            (new TestVertex(superstep < numSupersteps - 1, self.age + 1), Array[TestMessage]())
+        }
+      for ((id, vert) <- result.collect) {
+        assert(vert.age === numSupersteps)
+      }
+    }
   }
 }
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 a901b1c26068e47ad0eb476aacf4928b0124c0b9..83a3e1f3dc1a3caa04b83d60d9dd78f506db583c 100755
--- a/bin/start-master.sh
+++ b/bin/start-master.sh
@@ -26,9 +26,10 @@ fi
 # Set SPARK_PUBLIC_DNS so the master report the correct webUI address to the slaves
 if [ "$SPARK_PUBLIC_DNS" = "" ]; then
     # If we appear to be running on EC2, use the public address by default:
-    if [[ `hostname` == *ec2.internal ]]; then
+    # NOTE: ec2-metadata is installed on Amazon Linux AMI. Check based on that and hostname
+    if command -v ec2-metadata > /dev/null || [[ `hostname` == *ec2.internal ]]; then
         export SPARK_PUBLIC_DNS=`wget -q -O - http://instance-data.ec2.internal/latest/meta-data/public-hostname`
     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 6316b28a7befdd4acf0768c39322cc21683655d2..7f5cffc8182f7382ead77a38d7c0f8254245a2ec 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.0-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -91,6 +91,10 @@
       <groupId>org.apache.mesos</groupId>
       <artifactId>mesos</artifactId>
     </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>org.scalatest</groupId>
@@ -102,6 +106,11 @@
       <artifactId>scalacheck_${scala.version}</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.easymock</groupId>
+      <artifactId>easymock</artifactId>
+      <scope>test</scope>
+    </dependency>
     <dependency>
       <groupId>com.novocode</groupId>
       <artifactId>junit-interface</artifactId>
@@ -167,11 +176,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-      <activation>
-        <property>
-          <name>!hadoopVersion</name>
-        </property>
-      </activation>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -224,12 +228,6 @@
     </profile>
     <profile>
       <id>hadoop2</id>
-      <activation>
-        <property>
-          <name>hadoopVersion</name>
-          <value>2</value>
-        </property>
-      </activation>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala
index b644aba5f869dc3daf8fe66b6626ee4832a9cb5a..57c6df35bed9f8fed4964b59e65d39c441d29dc6 100644
--- a/core/src/main/scala/spark/Accumulators.scala
+++ b/core/src/main/scala/spark/Accumulators.scala
@@ -25,8 +25,7 @@ class Accumulable[R, T] (
   extends Serializable {
   
   val id = Accumulators.newId
-  @transient
-  private var value_ = initialValue // Current value on master
+  @transient private var value_ = initialValue // Current value on master
   val zero = param.zero(initialValue)  // Zero value to be passed to workers
   var deserialized = false
 
diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
index 86432d0127e3e9a75d24177ebc8558d453e0ecc8..c27ed3640611948743dee57ad469dc851b3c8b75 100644
--- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
@@ -1,20 +1,22 @@
 package spark
 
+import executor.{ShuffleReadMetrics, TaskMetrics}
 import scala.collection.mutable.ArrayBuffer
 import scala.collection.mutable.HashMap
 
-import spark.storage.BlockManagerId
+import spark.storage.{DelegateBlockFetchTracker, BlockManagerId}
+import util.{CompletionIterator, TimedIterator}
 
 private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
-  override def fetch[K, V](shuffleId: Int, reduceId: Int) = {
+  override def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) = {
     logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId))
     val blockManager = SparkEnv.get.blockManager
-    
+
     val startTime = System.currentTimeMillis
     val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId)
     logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format(
       shuffleId, reduceId, System.currentTimeMillis - startTime))
-    
+
     val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]
     for (((address, size), index) <- statuses.zipWithIndex) {
       splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size))
@@ -45,6 +47,20 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
         }
       }
     }
-    blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock)
+
+    val blockFetcherItr = blockManager.getMultiple(blocksByAddress)
+    val itr = new TimedIterator(blockFetcherItr.flatMap(unpackBlock)) with DelegateBlockFetchTracker
+    itr.setDelegate(blockFetcherItr)
+    CompletionIterator[(K,V), Iterator[(K,V)]](itr, {
+      val shuffleMetrics = new ShuffleReadMetrics
+      shuffleMetrics.shuffleReadMillis = itr.getNetMillis
+      shuffleMetrics.remoteFetchTime = itr.remoteFetchTime
+      shuffleMetrics.fetchWaitTime = itr.fetchWaitTime
+      shuffleMetrics.remoteBytesRead = itr.remoteBytesRead
+      shuffleMetrics.totalBlocksFetched = itr.totalBlocks
+      shuffleMetrics.localBlocksFetched = itr.numLocalBlocks
+      shuffleMetrics.remoteBlocksFetched = itr.numRemoteBlocks
+      metrics.shuffleReadMetrics = Some(shuffleMetrics)
+    })
   }
 }
diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f7a2b7e8027ef16c608b9a55e87db4ad9d1139c2
--- /dev/null
+++ b/core/src/main/scala/spark/CacheManager.scala
@@ -0,0 +1,65 @@
+package spark
+
+import scala.collection.mutable.{ArrayBuffer, HashSet}
+import spark.storage.{BlockManager, StorageLevel}
+
+
+/** Spark class responsible for passing RDDs split contents to the BlockManager and making
+    sure a node doesn't load two copies of an RDD at once.
+  */
+private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
+  private val loading = new HashSet[String]
+
+  /** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */
+  def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel)
+      : Iterator[T] = {
+    val key = "rdd_%d_%d".format(rdd.id, split.index)
+    logInfo("Cache key is " + key)
+    blockManager.get(key) match {
+      case Some(cachedValues) =>
+        // Partition is in cache, so just return its values
+        logInfo("Found partition in cache!")
+        return cachedValues.asInstanceOf[Iterator[T]]
+
+      case None =>
+        // Mark the split as loading (unless someone else marks it first)
+        loading.synchronized {
+          if (loading.contains(key)) {
+            logInfo("Loading contains " + key + ", waiting...")
+            while (loading.contains(key)) {
+              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
+            // is for the RDD-level cache eviction policy if someone else has loaded the same RDD
+            // partition but we didn't want to make space for it. However, that case is unlikely
+            // because it's unlikely that two threads would work on the same RDD partition. One
+            // downside of the current code is that threads wait serially if this does happen.
+            blockManager.get(key) match {
+              case Some(values) =>
+                return values.asInstanceOf[Iterator[T]]
+              case None =>
+                logInfo("Whoever was loading " + key + " failed; we'll try it ourselves")
+                loading.add(key)
+            }
+          } else {
+            loading.add(key)
+          }
+        }
+        try {
+          // If we got here, we have to load the split
+          val elements = new ArrayBuffer[Any]
+          logInfo("Computing partition " + split)
+          elements ++= rdd.computeOrReadCheckpoint(split, context)
+          // Try to put this block in the blockManager
+          blockManager.put(key, elements, storageLevel, true)
+          return elements.iterator.asInstanceOf[Iterator[T]]
+        } finally {
+          loading.synchronized {
+            loading.remove(key)
+            loading.notifyAll()
+          }
+        }
+    }
+  }
+}
diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala
deleted file mode 100644
index a73438208aa29ef19aaa0ea10b0a34239c057b11..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/CacheTracker.scala
+++ /dev/null
@@ -1,232 +0,0 @@
-package spark
-
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
-
-import akka.actor._
-import scala.concurrent.Await
-import akka.pattern.ask
-import akka.remote._
-import scala.concurrent.duration.Duration
-import akka.util.Timeout
-import scala.concurrent.duration._
-
-import spark.storage.BlockManager
-import spark.storage.StorageLevel
-
-private[spark] sealed trait CacheTrackerMessage
-
-private[spark] case class AddedToCache(rddId: Int, partition: Int, host: String, size: Long = 0L)
-  extends CacheTrackerMessage
-private[spark] case class DroppedFromCache(rddId: Int, partition: Int, host: String, size: Long = 0L)
-  extends CacheTrackerMessage
-private[spark] case class MemoryCacheLost(host: String) extends CacheTrackerMessage
-private[spark] case class RegisterRDD(rddId: Int, numPartitions: Int) extends CacheTrackerMessage
-private[spark] case class SlaveCacheStarted(host: String, size: Long) extends CacheTrackerMessage
-private[spark] case object GetCacheStatus extends CacheTrackerMessage
-private[spark] case object GetCacheLocations extends CacheTrackerMessage
-private[spark] case object StopCacheTracker extends CacheTrackerMessage
-
-private[spark] class CacheTrackerActor extends Actor with Logging {
-  // TODO: Should probably store (String, CacheType) tuples
-  private val locs = new HashMap[Int, Array[List[String]]]
-
-  /**
-   * A map from the slave's host name to its cache size.
-   */
-  private val slaveCapacity = new HashMap[String, Long]
-  private val slaveUsage = new HashMap[String, Long]
-
-  private def getCacheUsage(host: String): Long = slaveUsage.getOrElse(host, 0L)
-  private def getCacheCapacity(host: String): Long = slaveCapacity.getOrElse(host, 0L)
-  private def getCacheAvailable(host: String): Long = getCacheCapacity(host) - getCacheUsage(host)
-
-  def receive = {
-    case SlaveCacheStarted(host: String, size: Long) =>
-      slaveCapacity.put(host, size)
-      slaveUsage.put(host, 0)
-      sender ! true
-
-    case RegisterRDD(rddId: Int, numPartitions: Int) =>
-      logInfo("Registering RDD " + rddId + " with " + numPartitions + " partitions")
-      locs(rddId) = Array.fill[List[String]](numPartitions)(Nil)
-      sender ! true
-
-    case AddedToCache(rddId, partition, host, size) =>
-      slaveUsage.put(host, getCacheUsage(host) + size)
-      locs(rddId)(partition) = host :: locs(rddId)(partition)
-      sender ! true
-
-    case DroppedFromCache(rddId, partition, host, size) =>
-      slaveUsage.put(host, getCacheUsage(host) - size)
-      // Do a sanity check to make sure usage is greater than 0.
-      locs(rddId)(partition) = locs(rddId)(partition).filterNot(_ == host)
-      sender ! true
-
-    case MemoryCacheLost(host) =>
-      logInfo("Memory cache lost on " + host)
-      for ((id, locations) <- locs) {
-        for (i <- 0 until locations.length) {
-          locations(i) = locations(i).filterNot(_ == host)
-        }
-      }
-      sender ! true
-
-    case GetCacheLocations =>
-      logInfo("Asked for current cache locations")
-      sender ! locs.map{case (rrdId, array) => (rrdId -> array.clone())}
-
-    case GetCacheStatus =>
-      val status = slaveCapacity.map { case (host, capacity) =>
-        (host, capacity, getCacheUsage(host))
-      }.toSeq
-      sender ! status
-
-    case StopCacheTracker =>
-      logInfo("Stopping CacheTrackerActor")
-      sender ! true
-      context.stop(self)
-  }
-}
-
-private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager)
-  extends Logging {
-
-  // Tracker actor on the master, or remote reference to it on workers
-  val ip: String = System.getProperty("spark.master.host", "localhost")
-  val port: Int = System.getProperty("spark.master.port", "7077").toInt
-  val actorName: String = "CacheTracker"
-
-  val timeout = 10.seconds
-
-  var trackerActor: ActorRef = if (isMaster) {
-    val actor = actorSystem.actorOf(Props[CacheTrackerActor], name = actorName)
-    logInfo("Registered CacheTrackerActor actor")
-    actor
-  } else {
-    val url = "akka://spark@%s:%s/user/%s".format(ip, port, actorName)
-    actorSystem.actorFor(url)
-  }
-
-  val registeredRddIds = new HashSet[Int]
-
-  // Remembers which splits are currently being loaded (on worker nodes)
-  val loading = new HashSet[String]
-
-  // Send a message to the trackerActor and get its result within a default timeout, or
-  // throw a SparkException if this fails.
-  def askTracker(message: Any): Any = {
-    try {
-      val future = trackerActor.ask(message)(timeout)
-      return Await.result(future, timeout)
-    } catch {
-      case e: Exception =>
-        throw new SparkException("Error communicating with CacheTracker", e)
-    }
-  }
-
-  // Send a one-way message to the trackerActor, to which we expect it to reply with true.
-  def communicate(message: Any) {
-    if (askTracker(message) != true) {
-      throw new SparkException("Error reply received from CacheTracker")
-    }
-  }
-
-  // Registers an RDD (on master only)
-  def registerRDD(rddId: Int, numPartitions: Int) {
-    registeredRddIds.synchronized {
-      if (!registeredRddIds.contains(rddId)) {
-        logInfo("Registering RDD ID " + rddId + " with cache")
-        registeredRddIds += rddId
-        communicate(RegisterRDD(rddId, numPartitions))
-      }
-    }
-  }
-
-  // For BlockManager.scala only
-  def cacheLost(host: String) {
-    communicate(MemoryCacheLost(host))
-    logInfo("CacheTracker successfully removed entries on " + host)
-  }
-
-  // Get the usage status of slave caches. Each tuple in the returned sequence
-  // is in the form of (host name, capacity, usage).
-  def getCacheStatus(): Seq[(String, Long, Long)] = {
-    askTracker(GetCacheStatus).asInstanceOf[Seq[(String, Long, Long)]]
-  }
-
-  // For BlockManager.scala only
-  def notifyFromBlockManager(t: AddedToCache) {
-    communicate(t)
-  }
-
-  // Get a snapshot of the currently known locations
-  def getLocationsSnapshot(): HashMap[Int, Array[List[String]]] = {
-    askTracker(GetCacheLocations).asInstanceOf[HashMap[Int, Array[List[String]]]]
-  }
-
-  // Gets or computes an RDD split
-  def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel)
-  : Iterator[T] = {
-    val key = "rdd_%d_%d".format(rdd.id, split.index)
-    logInfo("Cache key is " + key)
-    blockManager.get(key) match {
-      case Some(cachedValues) =>
-        // Split is in cache, so just return its values
-        logInfo("Found partition in cache!")
-        return cachedValues.asInstanceOf[Iterator[T]]
-
-      case None =>
-        // Mark the split as loading (unless someone else marks it first)
-        loading.synchronized {
-          if (loading.contains(key)) {
-            logInfo("Loading contains " + key + ", waiting...")
-            while (loading.contains(key)) {
-              try {loading.wait()} catch {case _ =>}
-            }
-            logInfo("Loading no longer contains " + key + ", so returning cached result")
-            // See whether someone else has successfully loaded it. The main way this would fail
-            // is for the RDD-level cache eviction policy if someone else has loaded the same RDD
-            // partition but we didn't want to make space for it. However, that case is unlikely
-            // because it's unlikely that two threads would work on the same RDD partition. One
-            // downside of the current code is that threads wait serially if this does happen.
-            blockManager.get(key) match {
-              case Some(values) =>
-                return values.asInstanceOf[Iterator[T]]
-              case None =>
-                logInfo("Whoever was loading " + key + " failed; we'll try it ourselves")
-                loading.add(key)
-            }
-          } else {
-            loading.add(key)
-          }
-        }
-        try {
-          // If we got here, we have to load the split
-          val elements = new ArrayBuffer[Any]
-          logInfo("Computing partition " + split)
-          elements ++= rdd.compute(split, context)
-          // Try to put this block in the blockManager
-          blockManager.put(key, elements, storageLevel, true)
-          return elements.iterator.asInstanceOf[Iterator[T]]
-        } finally {
-          loading.synchronized {
-            loading.remove(key)
-            loading.notifyAll()
-          }
-        }
-    }
-  }
-
-  // Called by the Cache to report that an entry has been dropped from it
-  def dropEntry(rddId: Int, partition: Int) {
-    communicate(DroppedFromCache(rddId, partition, Utils.localHostName()))
-  }
-
-  def stop() {
-    communicate(StopCacheTracker)
-    registeredRddIds.clear()
-    trackerActor = null
-  }
-}
diff --git a/core/src/main/scala/spark/DaemonThreadFactory.scala b/core/src/main/scala/spark/DaemonThreadFactory.scala
deleted file mode 100644
index 56e59adeb7152b7d4e6ffe6bb2fef7d8c1dc4132..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/DaemonThreadFactory.scala
+++ /dev/null
@@ -1,18 +0,0 @@
-package spark
-
-import java.util.concurrent.ThreadFactory
-
-/**
- * A ThreadFactory that creates daemon threads
- */
-private object DaemonThreadFactory extends ThreadFactory {
-  override def newThread(r: Runnable): Thread = new DaemonThread(r)
-}
-
-private class DaemonThread(r: Runnable = null) extends Thread {
-  override def run() {
-    if (r != null) {
-      r.run()
-    }
-  }
-}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala
index b85d2732db91919ad708d94e4e3a73c85b9d5bbb..5eea9073220f1259c61f1082eb78996294fc434f 100644
--- a/core/src/main/scala/spark/Dependency.scala
+++ b/core/src/main/scala/spark/Dependency.scala
@@ -5,6 +5,7 @@ package spark
  */
 abstract class Dependency[T](val rdd: RDD[T]) extends Serializable
 
+
 /**
  * Base class for dependencies where each partition of the parent RDD is used by at most one
  * partition of the child RDD.  Narrow dependencies allow for pipelined execution.
@@ -12,12 +13,13 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable
 abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
   /**
    * Get the parent partitions for a child partition.
-   * @param outputPartition a partition of the child RDD
+   * @param partitionId a partition of the child RDD
    * @return the partitions of the parent RDD that the child partition depends upon
    */
-  def getParents(outputPartition: Int): Seq[Int]
+  def getParents(partitionId: Int): Seq[Int]
 }
 
+
 /**
  * Represents a dependency on the output of a shuffle stage.
  * @param shuffleId the shuffle id
@@ -32,6 +34,7 @@ class ShuffleDependency[K, V](
   val shuffleId: Int = rdd.context.newShuffleId()
 }
 
+
 /**
  * Represents a one-to-one dependency between partitions of the parent and child RDDs.
  */
@@ -39,6 +42,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
   override def getParents(partitionId: Int) = List(partitionId)
 }
 
+
 /**
  * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs.
  * @param rdd the parent RDD
@@ -48,7 +52,7 @@ class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
  */
 class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int)
   extends NarrowDependency[T](rdd) {
-  
+
   override def getParents(partitionId: Int) = {
     if (partitionId >= outStart && partitionId < outStart + length) {
       List(partitionId - outStart + inStart)
diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala
index b2a0e2b631e7aabcd8892a93aec2d0bbbbb28a35..178d31a73b9130fce8d96bc57ca0c1f82a4876a8 100644
--- a/core/src/main/scala/spark/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala
@@ -42,14 +42,14 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
   /** (Experimental) Approximate operation to return the mean within a timeout. */
   def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
     val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
-    val evaluator = new MeanEvaluator(self.splits.size, confidence)
+    val evaluator = new MeanEvaluator(self.partitions.size, confidence)
     self.context.runApproximateJob(self, processPartition, evaluator, timeout)
   }
 
   /** (Experimental) Approximate operation to return the sum within a timeout. */
   def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
     val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
-    val evaluator = new SumEvaluator(self.splits.size, confidence)
+    val evaluator = new SumEvaluator(self.partitions.size, confidence)
     self.context.runApproximateJob(self, processPartition, evaluator, timeout)
   }
 }
diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala
index 659d17718fae409b2f8d3d87e02068d174b7328e..00901d95e26c3d958b6ed2c64f7d6fc3be6d2d8b 100644
--- a/core/src/main/scala/spark/HttpFileServer.scala
+++ b/core/src/main/scala/spark/HttpFileServer.scala
@@ -1,9 +1,7 @@
 package spark
 
-import java.io.{File, PrintWriter}
-import java.net.URL
-import scala.collection.mutable.HashMap
-import org.apache.hadoop.fs.FileUtil
+import java.io.{File}
+import com.google.common.io.Files
 
 private[spark] class HttpFileServer extends Logging {
   
@@ -40,7 +38,7 @@ private[spark] class HttpFileServer extends Logging {
   }
   
   def addFileToDir(file: File, dir: File) : String = {
-    Utils.copyFile(file, new File(dir, file.getName))
+    Files.copy(file, new File(dir, file.getName))
     return dir + "/" + file.getName
   }
   
diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala
index 0196595ba1cfa589998094fff327f4872ef7c3da..4e0507c080adf2a649c333cf02e3cea9fd6b8496 100644
--- a/core/src/main/scala/spark/HttpServer.scala
+++ b/core/src/main/scala/spark/HttpServer.scala
@@ -4,6 +4,7 @@ import java.io.File
 import java.net.InetAddress
 
 import org.eclipse.jetty.server.Server
+import org.eclipse.jetty.server.bio.SocketConnector
 import org.eclipse.jetty.server.handler.DefaultHandler
 import org.eclipse.jetty.server.handler.HandlerList
 import org.eclipse.jetty.server.handler.ResourceHandler
@@ -27,7 +28,13 @@ private[spark] class HttpServer(resourceBase: File) extends Logging {
     if (server != null) {
       throw new ServerStateException("Server is already started")
     } else {
-      server = new Server(0)
+      server = new Server()
+      val connector = new SocketConnector
+      connector.setMaxIdleTime(60*1000)
+      connector.setSoLingerTime(-1)
+      connector.setPort(0)
+      server.addConnector(connector)
+
       val threadPool = new QueuedThreadPool
       threadPool.setDaemon(true)
       server.setThreadPool(threadPool)
diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala
index 93d7327324ed3f43a7cbf6947d2ec30c0f8d0c85..d723ab7b1ec67e8da81f92acfc0d3cfcce2e3843 100644
--- a/core/src/main/scala/spark/KryoSerializer.scala
+++ b/core/src/main/scala/spark/KryoSerializer.scala
@@ -157,27 +157,34 @@ class KryoSerializer extends spark.serializer.Serializer with Logging {
 
     // Register maps with a special serializer since they have complex internal structure
     class ScalaMapSerializer(buildMap: Array[(Any, Any)] => scala.collection.Map[Any, Any])
-    extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] {
+      extends KSerializer[Array[(Any, Any)] => scala.collection.Map[Any, Any]] {
+
+      //hack, look at https://groups.google.com/forum/#!msg/kryo-users/Eu5V4bxCfws/k-8UQ22y59AJ
+      private final val FAKE_REFERENCE = new Object()
       override def write(
-        kryo: Kryo,
-        output: KryoOutput,
-        obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) {
+                          kryo: Kryo,
+                          output: KryoOutput,
+                          obj: Array[(Any, Any)] => scala.collection.Map[Any, Any]) {
         val map = obj.asInstanceOf[scala.collection.Map[Any, Any]]
-        kryo.writeObject(output, map.size.asInstanceOf[java.lang.Integer])
+        output.writeInt(map.size)
         for ((k, v) <- map) {
           kryo.writeClassAndObject(output, k)
           kryo.writeClassAndObject(output, v)
         }
       }
       override def read (
-        kryo: Kryo,
-        input: KryoInput,
-        cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]])
+                          kryo: Kryo,
+                          input: KryoInput,
+                          cls: Class[Array[(Any, Any)] => scala.collection.Map[Any, Any]])
       : Array[(Any, Any)] => scala.collection.Map[Any, Any] = {
-        val size = kryo.readObject(input, classOf[java.lang.Integer]).intValue
+        kryo.reference(FAKE_REFERENCE)
+        val size = input.readInt()
         val elems = new Array[(Any, Any)](size)
-        for (i <- 0 until size)
-          elems(i) = (kryo.readClassAndObject(input), kryo.readClassAndObject(input))
+        for (i <- 0 until size) {
+          val k = kryo.readClassAndObject(input)
+          val v = kryo.readClassAndObject(input)
+          elems(i)=(k,v)
+        }
         buildMap(elems).asInstanceOf[Array[(Any, Any)] => scala.collection.Map[Any, Any]]
       }
     }
@@ -206,5 +213,8 @@ class KryoSerializer extends spark.serializer.Serializer with Logging {
     kryo
   }
 
-  def newInstance(): SerializerInstance = new KryoSerializerInstance(this)
+  def newInstance(): SerializerInstance = {
+    this.kryo.get().setClassLoader(Thread.currentThread().getContextClassLoader)
+    new KryoSerializerInstance(this)
+  }
 }
diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala
index 90bae26202bd93033c338fc2f9d00aa06ade853e..7c1c1bb1440bcbd263947481ee8a0d762d1587be 100644
--- a/core/src/main/scala/spark/Logging.scala
+++ b/core/src/main/scala/spark/Logging.scala
@@ -11,8 +11,7 @@ import org.slf4j.LoggerFactory
 trait Logging {
   // Make the log field transient so that objects with Logging can
   // be serialized and used on another machine
-  @transient
-  private var log_ : Logger = null
+  @transient private var log_ : Logger = null
 
   // Method to get or create the logger for this object
   protected def log: Logger = {
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala
index b2c80d8eff8fad3a12f98a3f1a6acfb6e542be06..50708d9cb11113646ff64e01eff5c8a9c36de676 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/spark/MapOutputTracker.scala
@@ -17,6 +17,7 @@ import scala.concurrent.duration._
 
 import spark.scheduler.MapStatus
 import spark.storage.BlockManagerId
+import spark.util.{MetadataCleaner, TimeStampedHashMap}
 
 
 private[spark] sealed trait MapOutputTrackerMessage
@@ -37,14 +38,12 @@ private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Ac
   }
 }
 
-private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logging {
-  val ip: String = System.getProperty("spark.master.host", "localhost")
-  val port: Int = System.getProperty("spark.master.port", "7077").toInt
-  val actorName: String = "MapOutputTracker"
+private[spark] class MapOutputTracker extends Logging {
 
-  val timeout = 10.seconds
+  // Set to the MapOutputTrackerActor living on the driver
+  var trackerActor: ActorRef = _
 
-  var mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
+  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.
@@ -53,21 +52,15 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
 
   // Cache a serialized version of the output statuses for each shuffle to send them out faster
   var cacheGeneration = generation
-  val cachedSerializedStatuses = new HashMap[Int, Array[Byte]]
-
-  var trackerActor: ActorRef = if (isMaster) {
-    val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName)
-    logInfo("Registered MapOutputTrackerActor actor")
-    actor
-  } else {
-    val url = "akka://spark@%s:%s/user/%s".format(ip, port, actorName)
-    actorSystem.actorFor(url)
-  }
+  val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
+
+  val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup)
 
   // Send a message to the trackerActor and get its result within a default timeout, or
   // throw a SparkException if this fails.
   def askTracker(message: Any): Any = {
     try {
+      val timeout = 10.seconds
       val future = trackerActor.ask(message)(timeout)
       return Await.result(future, timeout)
     } catch {
@@ -84,14 +77,14 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
   }
 
   def registerShuffle(shuffleId: Int, numMaps: Int) {
-    if (mapStatuses.get(shuffleId) != null) {
+    if (mapStatuses.get(shuffleId) != None) {
       throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
     }
     mapStatuses.put(shuffleId, new Array[MapStatus](numMaps))
   }
 
   def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
-    var array = mapStatuses.get(shuffleId)
+    var array = mapStatuses(shuffleId)
     array.synchronized {
       array(mapId) = status
     }
@@ -108,10 +101,10 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
   }
 
   def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
-    var array = mapStatuses.get(shuffleId)
+    var array = mapStatuses(shuffleId)
     if (array != null) {
       array.synchronized {
-        if (array(mapId) != null && array(mapId).address == bmAddress) {
+        if (array(mapId) != null && array(mapId).location == bmAddress) {
           array(mapId) = null
         }
       }
@@ -126,7 +119,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
 
   // 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)
+    val statuses = mapStatuses.get(shuffleId).orNull
     if (statuses == null) {
       logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
       fetching.synchronized {
@@ -139,8 +132,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
               case e: InterruptedException =>
             }
           }
-          return MapOutputTracker.convertMapStatuses(shuffleId, reduceId,
-                                                     mapStatuses.get(shuffleId))
+          return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, mapStatuses(shuffleId))
         } else {
           fetching += shuffleId
         }
@@ -168,9 +160,15 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
     }
   }
 
+  private def cleanup(cleanupTime: Long) {
+    mapStatuses.clearOldValues(cleanupTime)
+    cachedSerializedStatuses.clearOldValues(cleanupTime)
+  }
+
   def stop() {
     communicate(StopMapOutputTracker)
     mapStatuses.clear()
+    metadataCleaner.cancel()
     trackerActor = null
   }
 
@@ -196,7 +194,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
     generationLock.synchronized {
       if (newGen > generation) {
         logInfo("Updating generation to " + newGen + " and clearing cache")
-        mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
+        mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]
         generation = newGen
       }
     }
@@ -214,7 +212,7 @@ private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolea
         case Some(bytes) =>
           return bytes
         case None =>
-          statuses = mapStatuses.get(shuffleId)
+          statuses = mapStatuses(shuffleId)
           generationGotten = generation
       }
     }
@@ -269,7 +267,7 @@ private[spark] object MapOutputTracker {
           throw new FetchFailedException(null, shuffleId, -1, reduceId,
             new Exception("Missing an output location for shuffle " + shuffleId))
         } else {
-          (status.address, decompressSize(status.compressedSizes(reduceId)))
+          (status.location, decompressSize(status.compressedSizes(reduceId)))
         }
     }
   }
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index ce48cea903e6559a65d702cd2376e4553c909684..07efba9e8d26efa2e9a737f28d90e70a665f51d6 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -23,6 +23,7 @@ import spark.partial.BoundedDouble
 import spark.partial.PartialResult
 import spark.rdd._
 import spark.SparkContext._
+import spark.Partitioner._
 
 /**
  * Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
@@ -62,7 +63,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
     }
     val aggregator =
       new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
-    if (mapSideCombine) {
+    if (self.partitioner == Some(partitioner)) {
+      self.mapPartitions(aggregator.combineValuesByKey(_), true)
+    } else if (mapSideCombine) {
       val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true)
       val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner)
       partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true)
@@ -81,8 +84,35 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
   def combineByKey[C](createCombiner: V => C,
       mergeValue: (C, V) => C,
       mergeCombiners: (C, C) => C,
-      numSplits: Int): RDD[(K, C)] = {
-    combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits))
+      numPartitions: Int): RDD[(K, C)] = {
+    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)
   }
 
   /**
@@ -143,10 +173,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
   /**
    * 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
-   * "combiner" in MapReduce. Output will be hash-partitioned with numSplits splits.
+   * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
    */
-  def reduceByKey(func: (V, V) => V, numSplits: Int): RDD[(K, V)] = {
-    reduceByKey(new HashPartitioner(numSplits), func)
+  def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = {
+    reduceByKey(new HashPartitioner(numPartitions), func)
   }
 
   /**
@@ -164,10 +194,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
 
   /**
    * Group the values for each key in the RDD into a single sequence. Hash-partitions the
-   * resulting RDD with into `numSplits` partitions.
+   * resulting RDD with into `numPartitions` partitions.
    */
-  def groupByKey(numSplits: Int): RDD[(K, Seq[V])] = {
-    groupByKey(new HashPartitioner(numSplits))
+  def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = {
+    groupByKey(new HashPartitioner(numPartitions))
   }
 
   /**
@@ -199,9 +229,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
   }
 
   /**
-   * 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
-   * "combiner" in MapReduce.
+   * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+   * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+   * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD.
    */
   def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
     this.cogroup(other, partitioner).flatMapValues {
@@ -246,8 +276,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
   }
 
   /**
-   * Simplified version of combineByKey that hash-partitions the resulting RDD using the default
-   * parallelism level.
+   * Simplified version of combineByKey that hash-partitions the resulting RDD using the
+   * existing partitioner/parallelism level.
    */
   def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C)
       : RDD[(K, C)] = {
@@ -257,7 +287,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
   /**
    * 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
-   * "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level.
+   * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/
+   * parallelism level.
    */
   def reduceByKey(func: (V, V) => V): RDD[(K, V)] = {
     reduceByKey(defaultPartitioner(self), func)
@@ -265,7 +296,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
 
   /**
    * Group the values for each key in the RDD into a single sequence. Hash-partitions the
-   * resulting RDD with the default parallelism level.
+   * resulting RDD with the existing partitioner/parallelism level.
    */
   def groupByKey(): RDD[(K, Seq[V])] = {
     groupByKey(defaultPartitioner(self))
@@ -285,15 +316,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
    * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
    * (k, v2) is in `other`. Performs a hash join across the cluster.
    */
-  def join[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, W))] = {
-    join(other, new HashPartitioner(numSplits))
+  def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = {
+    join(other, new HashPartitioner(numPartitions))
   }
 
   /**
    * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
    * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
    * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
-   * using the default level of parallelism.
+   * using the existing partitioner/parallelism level.
    */
   def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = {
     leftOuterJoin(other, defaultPartitioner(self, other))
@@ -303,17 +334,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
    * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
    * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
    * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
-   * into `numSplits` partitions.
+   * into `numPartitions` partitions.
    */
-  def leftOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, Option[W]))] = {
-    leftOuterJoin(other, new HashPartitioner(numSplits))
+  def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = {
+    leftOuterJoin(other, new HashPartitioner(numPartitions))
   }
 
   /**
    * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
    * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
    * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
-   * RDD using the default parallelism level.
+   * RDD using the existing partitioner/parallelism level.
    */
   def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = {
     rightOuterJoin(other, defaultPartitioner(self, other))
@@ -325,8 +356,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
    * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
    * RDD into the given number of partitions.
    */
-  def rightOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (Option[V], W))] = {
-    rightOuterJoin(other, new HashPartitioner(numSplits))
+  def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = {
+    rightOuterJoin(other, new HashPartitioner(numPartitions))
   }
 
   /**
@@ -361,7 +392,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       throw new SparkException("Default partitioner cannot partition array keys.")
     }
     val cg = new CoGroupedRDD[K](
-        Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]),
+        Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]),
         partitioner)
     val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
     prfs.mapValues {
@@ -380,9 +411,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       throw new SparkException("Default partitioner cannot partition array keys.")
     }
     val cg = new CoGroupedRDD[K](
-        Seq(self.asInstanceOf[RDD[(_, _)]],
-            other1.asInstanceOf[RDD[(_, _)]],
-            other2.asInstanceOf[RDD[(_, _)]]),
+        Seq(self.asInstanceOf[RDD[(K, _)]],
+            other1.asInstanceOf[RDD[(K, _)]],
+            other2.asInstanceOf[RDD[(K, _)]]),
         partitioner)
     val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
     prfs.mapValues {
@@ -412,17 +443,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
    * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
    * list of values for that key in `this` as well as `other`.
    */
-  def cogroup[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (Seq[V], Seq[W]))] = {
-    cogroup(other, new HashPartitioner(numSplits))
+  def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = {
+    cogroup(other, new HashPartitioner(numPartitions))
   }
 
   /**
    * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
    * tuple with the list of values for that key in `this`, `other1` and `other2`.
    */
-  def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numSplits: Int)
+  def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int)
       : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = {
-    cogroup(other1, other2, new HashPartitioner(numSplits))
+    cogroup(other1, other2, new HashPartitioner(numPartitions))
   }
 
   /** Alias for cogroup. */
@@ -437,15 +468,21 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
   }
 
   /**
-   * Choose a partitioner to use for a cogroup-like operation between a number of RDDs. If any of
-   * the RDDs already has a partitioner, choose that one, otherwise use a default HashPartitioner.
+   * 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 defaultPartitioner(rdds: RDD[_]*): Partitioner = {
-    for (r <- rdds if r.partitioner != None) {
-      return r.partitioner.get
-    }
-    return new HashPartitioner(self.context.defaultParallelism)
-  }
+  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
@@ -465,7 +502,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
         val res = self.context.runJob(self, process _, Array(index), false)
         res(0)
       case None =>
-        self.filter(_._1 == key).map(_._2).collect
+        self.filter(_._1 == key).map(_._2).collect()
     }
   }
 
@@ -485,18 +522,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
     saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
   }
 
-  /**
-   * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
-   * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
-   */
-  def saveAsNewAPIHadoopFile(
-      path: String,
-      keyClass: Class[_],
-      valueClass: Class[_],
-      outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) {
-    saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, new Configuration)
-  }
-
   /**
    * Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
    * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
@@ -506,7 +531,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       keyClass: Class[_],
       valueClass: Class[_],
       outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
-      conf: Configuration) {
+      conf: Configuration = self.context.hadoopConfiguration) {
     val job = new NewAPIHadoopJob(conf)
     job.setOutputKeyClass(keyClass)
     job.setOutputValueClass(valueClass)
@@ -557,7 +582,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
       keyClass: Class[_],
       valueClass: Class[_],
       outputFormatClass: Class[_ <: OutputFormat[_, _]],
-      conf: JobConf = new JobConf) {
+      conf: JobConf = new JobConf(self.context.hadoopConfiguration)) {
     conf.setOutputKeyClass(keyClass)
     conf.setOutputValueClass(valueClass)
     // conf.setOutputFormat(outputFormatClass) // Doesn't work in Scala 2.9 due to what may be a generics bug
@@ -602,7 +627,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
 
       var count = 0
       while(iter.hasNext) {
-        val record = iter.next
+        val record = iter.next()
         count += 1
         writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef])
       }
@@ -646,9 +671,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
    * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
    * order of the keys).
    */
-  def sortByKey(ascending: Boolean = true, numSplits: Int = self.splits.size): RDD[(K,V)] = {
+  def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[(K,V)] = {
     val shuffled =
-      new ShuffledRDD[K, V](self, new RangePartitioner(numSplits, self, ascending))
+      new ShuffledRDD[K, V](self, new RangePartitioner(numPartitions, self, ascending))
     shuffled.mapPartitions(iter => {
       val buf = iter.toArray
       if (ascending) {
@@ -661,24 +686,21 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
 }
 
 private[spark]
-class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) {
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override val partitioner = prev.partitioner
-  override def compute(split: Split, taskContext: TaskContext) =
-    prev.iterator(split, taskContext).map{case (k, v) => (k, f(v))}
+class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev) {
+  override def getPartitions = firstParent[(K, V)].partitions
+  override val partitioner = firstParent[(K, V)].partitioner
+  override def compute(split: Partition, context: TaskContext) =
+    firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) }
 }
 
 private[spark]
 class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U])
-  extends RDD[(K, U)](prev.context) {
-
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override val partitioner = prev.partitioner
+  extends RDD[(K, U)](prev) {
 
-  override def compute(split: Split, taskContext: TaskContext) = {
-    prev.iterator(split, taskContext).flatMap { case (k, v) => f(v).map(x => (k, x)) }
+  override def getPartitions = firstParent[(K, V)].partitions
+  override val partitioner = firstParent[(K, V)].partitioner
+  override def compute(split: Partition, context: TaskContext) = {
+    firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) }
   }
 }
 
diff --git a/core/src/main/scala/spark/Split.scala b/core/src/main/scala/spark/Partition.scala
similarity index 84%
rename from core/src/main/scala/spark/Split.scala
rename to core/src/main/scala/spark/Partition.scala
index 90d4b47c553c535ff3271da0ddceb66a7ed832f9..e384308ef6e939c342f8e3f36af0115a87ac8838 100644
--- a/core/src/main/scala/spark/Split.scala
+++ b/core/src/main/scala/spark/Partition.scala
@@ -3,7 +3,7 @@ package spark
 /**
  * A partition of an RDD.
  */
-trait Split extends Serializable {
+trait Partition extends Serializable {
   /**
    * Get the split's index within its parent RDD
    */
diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala
index 9d5b966e1e5a092b542a2621aa2e91c17a9b299a..6f8cd17c88220a3e0b1a4f1bf77cbd1dad9ccc10 100644
--- a/core/src/main/scala/spark/Partitioner.scala
+++ b/core/src/main/scala/spark/Partitioner.scala
@@ -9,6 +9,35 @@ abstract class Partitioner extends Serializable {
   def getPartition(key: Any): Int
 }
 
+object Partitioner {
+  /**
+   * Choose a partitioner to use for a cogroup-like operation between a number of RDDs.
+   *
+   * If any of the RDDs already has a partitioner, choose that one.
+   *
+   * Otherwise, we use a default HashPartitioner. For the number of partitions, if
+   * spark.default.parallelism is set, then we'll use the value from SparkContext
+   * defaultParallelism, otherwise we'll use the max number of upstream partitions.
+   *
+   * Unless spark.default.parallelism is set, He number of partitions will be the
+   * same as the number of partitions in the largest upstream RDD, as this should
+   * be least likely to cause out-of-memory errors.
+   *
+   * We use two method parameters (rdd, others) to enforce callers passing at least 1 RDD.
+   */
+  def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = {
+    val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.size).reverse
+    for (r <- bySize if r.partitioner != None) {
+      return r.partitioner.get
+    }
+    if (System.getProperty("spark.default.parallelism") != null) {
+      return new HashPartitioner(rdd.context.defaultParallelism)
+    } else {
+      return new HashPartitioner(bySize.head.partitions.size)
+    }
+  }
+}
+
 /**
  * A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
  *
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index 3b9ced1946b25b14542844936e675b5c8e6faa76..ccd9d0364ad9582bf319dda96fb57566a5da60ae 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -1,12 +1,8 @@
 package spark
 
-import java.io.EOFException
-import java.io.ObjectInputStream
 import java.net.URL
-import java.util.Random
-import java.util.Date
+import java.util.{Date, Random}
 import java.util.{HashMap => JHashMap}
-import java.util.concurrent.atomic.AtomicLong
 
 import scala.collection.Map
 import scala.collection.JavaConversions.mapAsScalaMap
@@ -16,31 +12,27 @@ import scala.collection.mutable.HashMap
 import org.apache.hadoop.io.BytesWritable
 import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.Writable
-import org.apache.hadoop.mapred.FileOutputCommitter
-import org.apache.hadoop.mapred.HadoopWriter
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapred.OutputCommitter
-import org.apache.hadoop.mapred.OutputFormat
-import org.apache.hadoop.mapred.SequenceFileOutputFormat
 import org.apache.hadoop.mapred.TextOutputFormat
 
 import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
 
+import spark.Partitioner._
 import spark.partial.BoundedDouble
 import spark.partial.CountEvaluator
 import spark.partial.GroupedCountEvaluator
 import spark.partial.PartialResult
-import spark.rdd.BlockRDD
+import spark.rdd.CoalescedRDD
 import spark.rdd.CartesianRDD
 import spark.rdd.FilteredRDD
 import spark.rdd.FlatMappedRDD
 import spark.rdd.GlommedRDD
 import spark.rdd.MappedRDD
 import spark.rdd.MapPartitionsRDD
-import spark.rdd.MapPartitionsWithSplitRDD
+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.storage.StorageLevel
@@ -60,7 +52,7 @@ import SparkContext._
  *
  * Internally, each RDD is characterized by five main properties:
  *
- *  - A list of splits (partitions)
+ *  - A list of partitions
  *  - A function for computing each split
  *  - A list of dependencies on other RDDs
  *  - Optionally, a Partitioner for key-value RDDs (e.g. to say that the RDD is hash-partitioned)
@@ -73,40 +65,55 @@ import SparkContext._
  * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details
  * on RDD internals.
  */
-abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serializable {
+abstract class RDD[T: ClassManifest](
+    @transient private var sc: SparkContext,
+    @transient private var deps: Seq[Dependency[_]]
+  ) extends Serializable with Logging {
 
-  // Methods that must be implemented by subclasses:
+  /** Construct an RDD with just a one-to-one dependency on one parent */
+  def this(@transient oneParent: RDD[_]) =
+    this(oneParent.context , List(new OneToOneDependency(oneParent)))
 
-  /** Set of partitions in this RDD. */
-  def splits: Array[Split]
+  // =======================================================================
+  // Methods that should be implemented by subclasses of RDD
+  // =======================================================================
 
-  /** Function for computing a given partition. */
-  def compute(split: Split, context: TaskContext): Iterator[T]
+  /** Implemented by subclasses to compute a given partition. */
+  def compute(split: Partition, context: TaskContext): Iterator[T]
 
-  /** How this RDD depends on any parent RDDs. */
-  @transient val dependencies: List[Dependency[_]]
+  /**
+   * Implemented by subclasses to return the set of partitions in this RDD. This method will only
+   * be called once, so it is safe to implement a time-consuming computation in it.
+   */
+  protected def getPartitions: Array[Partition]
 
-  // Methods available on all RDDs:
+  /**
+   * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only
+   * be called once, so it is safe to implement a time-consuming computation in it.
+   */
+  protected def getDependencies: Seq[Dependency[_]] = deps
 
-  /** Record user function generating this RDD. */
-  private[spark] val origin = Utils.getSparkCallSite
+  /** Optionally overridden by subclasses to specify placement preferences. */
+  protected def getPreferredLocations(split: Partition): Seq[String] = Nil
 
   /** Optionally overridden by subclasses to specify how they are partitioned. */
   val partitioner: Option[Partitioner] = None
 
-  /** Optionally overridden by subclasses to specify placement preferences. */
-  def preferredLocations(split: Split): Seq[String] = Nil
-
-  /** The [[spark.SparkContext]] that this RDD was created on. */
-  def context = sc
-
-  private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
+  // =======================================================================
+  // Methods and fields available on all RDDs
+  // =======================================================================
 
   /** A unique ID for this RDD (within its SparkContext). */
   val id = sc.newRddId()
 
-  // Variables relating to persistence
-  private var storageLevel: StorageLevel = StorageLevel.NONE
+  /** A friendly name for this RDD */
+  var name: String = null
+
+  /** Assign a name to this RDD */
+  def setName(_name: String) = {
+    name = _name
+    this
+  }
 
   /**
    * Set this RDD's storage level to persist its values across operations after the first time
@@ -119,6 +126,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
         "Cannot change storage level of an RDD after it was already assigned a level")
     }
     storageLevel = newLevel
+    // Register the RDD with the SparkContext
+    sc.persistentRdds(id) = this
     this
   }
 
@@ -131,22 +140,47 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
   /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
   def getStorageLevel = storageLevel
 
-  private[spark] def checkpoint(level: StorageLevel = StorageLevel.MEMORY_AND_DISK_2): RDD[T] = {
-    if (!level.useDisk && level.replication < 2) {
-      throw new Exception("Cannot checkpoint without using disk or replication (level requested was " + level + ")")
-    }
+  // Our dependencies and partitions will be gotten by calling subclass's methods below, and will
+  // be overwritten when we're checkpointed
+  private var dependencies_ : Seq[Dependency[_]] = null
+  @transient private var partitions_ : Array[Partition] = null
 
-    // This is a hack. Ideally this should re-use the code used by the CacheTracker
-    // to generate the key.
-    def getSplitKey(split: Split) = "rdd_%d_%d".format(this.id, split.index)
+  /** An Option holding our checkpoint RDD, if we are checkpointed */
+  private def checkpointRDD: Option[RDD[T]] = checkpointData.flatMap(_.checkpointRDD)
 
-    persist(level)
-    sc.runJob(this, (iter: Iterator[T]) => {} )
+  /**
+   * Get the list of dependencies of this RDD, taking into account whether the
+   * RDD is checkpointed or not.
+   */
+  final def dependencies: Seq[Dependency[_]] = {
+    checkpointRDD.map(r => List(new OneToOneDependency(r))).getOrElse {
+      if (dependencies_ == null) {
+        dependencies_ = getDependencies
+      }
+      dependencies_
+    }
+  }
 
-    val p = this.partitioner
+  /**
+   * Get the array of partitions of this RDD, taking into account whether the
+   * RDD is checkpointed or not.
+   */
+  final def partitions: Array[Partition] = {
+    checkpointRDD.map(_.partitions).getOrElse {
+      if (partitions_ == null) {
+        partitions_ = getPartitions
+      }
+      partitions_
+    }
+  }
 
-    new BlockRDD[T](sc, splits.map(getSplitKey).toArray) {
-      override val partitioner = p
+  /**
+   * Get the preferred location of a split, taking into account whether the
+   * RDD is checkpointed or not.
+   */
+  final def preferredLocations(split: Partition): Seq[String] = {
+    checkpointRDD.map(_.getPreferredLocations(split)).getOrElse {
+      getPreferredLocations(split)
     }
   }
 
@@ -155,9 +189,20 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
    * This should ''not'' be called by users directly, but is available for implementors of custom
    * subclasses of RDD.
    */
-  final def iterator(split: Split, context: TaskContext): Iterator[T] = {
+  final def iterator(split: Partition, context: TaskContext): Iterator[T] = {
     if (storageLevel != StorageLevel.NONE) {
-      SparkEnv.get.cacheTracker.getOrCompute[T](this, split, context, storageLevel)
+      SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel)
+    } else {
+      computeOrReadCheckpoint(split, context)
+    }
+  }
+
+  /**
+   * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing.
+   */
+  private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = {
+    if (isCheckpointed) {
+      firstParent[T].iterator(split, context)
     } else {
       compute(split, context)
     }
@@ -185,10 +230,22 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
   /**
    * Return a new RDD containing the distinct elements in this RDD.
    */
-  def distinct(numSplits: Int): RDD[T] =
-    map(x => (x, null)).reduceByKey((x, y) => x, numSplits).map(_._1)
+  def distinct(numPartitions: Int): RDD[T] =
+    map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1)
 
-  def distinct(): RDD[T] = distinct(splits.size)
+  def distinct(): RDD[T] = distinct(partitions.size)
+
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  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.
@@ -252,19 +309,26 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
    */
   def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other)
 
+  /**
+   * Return an RDD of grouped items.
+   */
+  def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] =
+    groupBy[K](f, defaultPartitioner(this))
+
   /**
    * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
    * mapping to that key.
    */
-  def groupBy[K: ClassManifest](f: T => K, numSplits: Int): RDD[(K, Seq[T])] = {
-    val cleanF = sc.clean(f)
-    this.map(t => (cleanF(t), t)).groupByKey(numSplits)
-  }
+  def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] =
+    groupBy(f, new HashPartitioner(numPartitions))
 
   /**
    * Return an RDD of grouped items.
    */
-  def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = groupBy[K](f, sc.defaultParallelism)
+  def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = {
+    val cleanF = sc.clean(f)
+    this.map(t => (cleanF(t), t)).groupByKey(p)
+  }
 
   /**
    * Return an RDD created by piping elements to a forked external process.
@@ -289,14 +353,80 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
     preservesPartitioning: Boolean = false): RDD[U] =
     new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
 
-   /**
+  /**
+   * Return a new RDD by applying a function to each partition of this RDD, while tracking the index
+   * of the original partition.
+   */
+  def mapPartitionsWithIndex[U: ClassManifest](
+    f: (Int, Iterator[T]) => Iterator[U],
+    preservesPartitioning: Boolean = false): RDD[U] =
+    new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
+
+  /**
    * 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", "0.7.0")
   def mapPartitionsWithSplit[U: ClassManifest](
     f: (Int, Iterator[T]) => Iterator[U],
     preservesPartitioning: Boolean = false): RDD[U] =
-    new MapPartitionsWithSplitRDD(this, sc.clean(f), preservesPartitioning)
+    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,
@@ -316,6 +446,14 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
     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]) => f(iter))
+  }
+
   /**
    * Return an array that contains all of the elements in this RDD.
    */
@@ -337,27 +475,65 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
   }
 
   /**
-   * Reduces the elements of this RDD using the specified associative binary operator.
+   * 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.
+   */
+  def subtract(other: RDD[T]): RDD[T] =
+    subtract(other, partitioner.getOrElse(new HashPartitioner(partitions.size)))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  def subtract(other: RDD[T], numPartitions: Int): RDD[T] =
+    subtract(other, new HashPartitioner(numPartitions))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  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.
    */
   def reduce(f: (T, T) => T): T = {
     val cleanF = sc.clean(f)
     val reducePartition: Iterator[T] => Option[T] = iter => {
       if (iter.hasNext) {
         Some(iter.reduceLeft(cleanF))
-      }else {
+      } else {
         None
       }
     }
-    val options = sc.runJob(this, reducePartition)
-    val results = new ArrayBuffer[T]
-    for (opt <- options; elem <- opt) {
-      results += elem
-    }
-    if (results.size == 0) {
-      throw new UnsupportedOperationException("empty collection")
-    } else {
-      return results.reduceLeft(cleanF)
+    var jobResult: Option[T] = None
+    val mergeResult = (index: Int, taskResult: Option[T]) => {
+      if (taskResult != None) {
+        jobResult = jobResult match {
+          case Some(value) => Some(f(value, taskResult.get))
+          case None => taskResult
+        }
+      }
     }
+    sc.runJob(this, reducePartition, mergeResult)
+    // Get the final result out of our Option, or throw an exception if the RDD was empty
+    jobResult.getOrElse(throw new UnsupportedOperationException("empty collection"))
   }
 
   /**
@@ -367,9 +543,13 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
    * modify t2.
    */
   def fold(zeroValue: T)(op: (T, T) => T): T = {
+    // Clone the zero value since we will also be serializing it as part of tasks
+    var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance())
     val cleanOp = sc.clean(op)
-    val results = sc.runJob(this, (iter: Iterator[T]) => iter.fold(zeroValue)(cleanOp))
-    return results.fold(zeroValue)(cleanOp)
+    val foldPartition = (iter: Iterator[T]) => iter.fold(zeroValue)(cleanOp)
+    val mergeResult = (index: Int, taskResult: T) => jobResult = op(jobResult, taskResult)
+    sc.runJob(this, foldPartition, mergeResult)
+    jobResult
   }
 
   /**
@@ -381,11 +561,14 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
    * allocation.
    */
   def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = {
+    // Clone the zero value since we will also be serializing it as part of tasks
+    var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance())
     val cleanSeqOp = sc.clean(seqOp)
     val cleanCombOp = sc.clean(combOp)
-    val results = sc.runJob(this,
-        (iter: Iterator[T]) => iter.aggregate(zeroValue)(cleanSeqOp, cleanCombOp))
-    return results.fold(zeroValue)(cleanCombOp)
+    val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp)
+    val mergeResult = (index: Int, taskResult: U) => jobResult = combOp(jobResult, taskResult)
+    sc.runJob(this, aggregatePartition, mergeResult)
+    jobResult
   }
 
   /**
@@ -396,7 +579,7 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
       var result = 0L
       while (iter.hasNext) {
         result += 1L
-        iter.next
+        iter.next()
       }
       result
     }).sum
@@ -411,11 +594,11 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
       var result = 0L
       while (iter.hasNext) {
         result += 1L
-        iter.next
+        iter.next()
       }
       result
     }
-    val evaluator = new CountEvaluator(splits.size, confidence)
+    val evaluator = new CountEvaluator(partitions.size, confidence)
     sc.runApproximateJob(this, countElements, evaluator, timeout)
   }
 
@@ -466,7 +649,7 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
       }
       map
     }
-    val evaluator = new GroupedCountEvaluator[T](splits.size, confidence)
+    val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence)
     sc.runApproximateJob(this, countPartition, evaluator, timeout)
   }
 
@@ -481,7 +664,7 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
     }
     val buf = new ArrayBuffer[T]
     var p = 0
-    while (buf.size < num && p < splits.size) {
+    while (buf.size < num && p < partitions.size) {
       val left = num - buf.size
       val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true)
       buf ++= res(0)
@@ -528,4 +711,110 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
   private[spark] def collectPartitions(): Array[Array[T]] = {
     sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
   }
+
+  /**
+   * Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint
+   * directory set with SparkContext.setCheckpointDir() and all references to its parent
+   * RDDs will be removed. This function must be called before any job has been
+   * executed on this RDD. It is strongly recommended that this RDD is persisted in
+   * memory, otherwise saving it on a file will require recomputation.
+   */
+  def checkpoint() {
+    if (context.checkpointDir.isEmpty) {
+      throw new Exception("Checkpoint directory has not been set in the SparkContext")
+    } else if (checkpointData.isEmpty) {
+      checkpointData = Some(new RDDCheckpointData(this))
+      checkpointData.get.markForCheckpoint()
+    }
+  }
+
+  /**
+   * Return whether this RDD has been checkpointed or not
+   */
+  def isCheckpointed: Boolean = {
+    checkpointData.map(_.isCheckpointed).getOrElse(false)
+  }
+
+  /**
+   * Gets the name of the file to which this RDD was checkpointed
+   */
+  def getCheckpointFile: Option[String] = {
+    checkpointData.flatMap(_.getCheckpointFile)
+  }
+
+  // =======================================================================
+  // Other internal methods and fields
+  // =======================================================================
+
+  private var storageLevel: StorageLevel = StorageLevel.NONE
+
+  /** Record user function generating this RDD. */
+  private[spark] val origin = Utils.getSparkCallSite
+
+  private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
+
+  private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None
+
+  /** Returns the first parent RDD */
+  protected[spark] def firstParent[U: ClassManifest] = {
+    dependencies.head.rdd.asInstanceOf[RDD[U]]
+  }
+
+  /** The [[spark.SparkContext]] that this RDD was created on. */
+  def context = sc
+
+  // Avoid handling doCheckpoint multiple times to prevent excessive recursion
+  private var doCheckpointCalled = false
+
+  /**
+   * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler
+   * after a job using this RDD has completed (therefore the RDD has been materialized and
+   * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs.
+   */
+  private[spark] def doCheckpoint() {
+    if (!doCheckpointCalled) {
+      doCheckpointCalled = true
+      if (checkpointData.isDefined) {
+        checkpointData.get.doCheckpoint()
+      } else {
+        dependencies.foreach(_.rdd.doCheckpoint())
+      }
+    }
+  }
+
+  /**
+   * Changes the dependencies of this RDD from its original parents to a new RDD (`newRDD`)
+   * created from the checkpoint file, and forget its old dependencies and partitions.
+   */
+  private[spark] def markCheckpointed(checkpointRDD: RDD[_]) {
+    clearDependencies()
+    partitions_ = null
+    deps = null    // Forget the constructor argument for dependencies too
+  }
+
+  /**
+   * Clears the dependencies of this RDD. This method must ensure that all references
+   * to the original parent RDDs is removed to enable the parent RDDs to be garbage
+   * collected. Subclasses of RDD may override this method for implementing their own cleaning
+   * logic. See [[spark.rdd.UnionRDD]] for an example.
+   */
+  protected def clearDependencies() {
+    dependencies_ = null
+  }
+
+  /** A description of this RDD and its recursive dependencies for debugging. */
+  def toDebugString: String = {
+    def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = {
+      Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++
+        rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + "  "))
+    }
+    debugString(this).mkString("\n")
+  }
+
+  override def toString: String = "%s%s[%d] at %s".format(
+    Option(name).map(_ + " ").getOrElse(""),
+    getClass.getSimpleName,
+    id,
+    origin)
+
 }
diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala
new file mode 100644
index 0000000000000000000000000000000000000000..d00092e9845e2a888de7d7ca03ba23d90ae50ba9
--- /dev/null
+++ b/core/src/main/scala/spark/RDDCheckpointData.scala
@@ -0,0 +1,106 @@
+package spark
+
+import org.apache.hadoop.fs.Path
+import rdd.{CheckpointRDD, CoalescedRDD}
+import scheduler.{ResultTask, ShuffleMapTask}
+
+/**
+ * Enumeration to manage state transitions of an RDD through checkpointing
+ * [ Initialized --> marked for checkpointing --> checkpointing in progress --> checkpointed ]
+ */
+private[spark] object CheckpointState extends Enumeration {
+  type CheckpointState = Value
+  val Initialized, MarkedForCheckpoint, CheckpointingInProgress, Checkpointed = Value
+}
+
+/**
+ * This class contains all the information related to RDD checkpointing. Each instance of this class
+ * is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as,
+ * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations
+ * of the checkpointed RDD.
+ */
+private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
+  extends Logging with Serializable {
+
+  import CheckpointState._
+
+  // The checkpoint state of the associated RDD.
+  var cpState = Initialized
+
+  // The file to which the associated RDD has been checkpointed to
+  @transient var cpFile: Option[String] = None
+
+  // The CheckpointRDD created from the checkpoint file, that is, the new parent the associated RDD.
+  var cpRDD: Option[RDD[T]] = None
+
+  // Mark the RDD for checkpointing
+  def markForCheckpoint() {
+    RDDCheckpointData.synchronized {
+      if (cpState == Initialized) cpState = MarkedForCheckpoint
+    }
+  }
+
+  // Is the RDD already checkpointed
+  def isCheckpointed: Boolean = {
+    RDDCheckpointData.synchronized { cpState == Checkpointed }
+  }
+
+  // Get the file to which this RDD was checkpointed to as an Option
+  def getCheckpointFile: Option[String] = {
+    RDDCheckpointData.synchronized { cpFile }
+  }
+
+  // Do the checkpointing of the RDD. Called after the first job using that RDD is over.
+  def doCheckpoint() {
+    // If it is marked for checkpointing AND checkpointing is not already in progress,
+    // then set it to be in progress, else return
+    RDDCheckpointData.synchronized {
+      if (cpState == MarkedForCheckpoint) {
+        cpState = CheckpointingInProgress
+      } else {
+        return
+      }
+    }
+
+    // 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)
+
+    // Change the dependencies and partitions of the RDD
+    RDDCheckpointData.synchronized {
+      cpFile = Some(path)
+      cpRDD = Some(newRDD)
+      rdd.markCheckpointed(newRDD)   // Update the RDD's dependencies and partitions
+      cpState = Checkpointed
+      RDDCheckpointData.clearTaskCaches()
+      logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id)
+    }
+  }
+
+  // Get preferred location of a split after checkpointing
+  def getPreferredLocations(split: Partition): Seq[String] = {
+    RDDCheckpointData.synchronized {
+      cpRDD.get.preferredLocations(split)
+    }
+  }
+
+  def getPartitions: Array[Partition] = {
+    RDDCheckpointData.synchronized {
+      cpRDD.get.partitions
+    }
+  }
+
+  def checkpointRDD: Option[RDD[T]] = {
+    RDDCheckpointData.synchronized {
+      cpRDD
+    }
+  }
+}
+
+private[spark] object RDDCheckpointData {
+  def clearTaskCaches() {
+    ShuffleMapTask.clearCache()
+    ResultTask.clearCache()
+  }
+}
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 d9a94d4021ee325d57aac710f4b0858883f3e63b..442e9f0269dc48284c8e20411e1f72427f627b96 100644
--- a/core/src/main/scala/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/spark/ShuffleFetcher.scala
@@ -1,11 +1,13 @@
 package spark
 
+import executor.TaskMetrics
+
 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) : Iterator[(K, V)]
+  def fetch[K, V](shuffleId: Int, reduceId: Int, metrics: TaskMetrics) : 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 bbf8272eb39e103b26ac29f4af05fe5b0c124c0e..4957a54c1b8af5c199a3f7ffd235c1ff492e0033 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -2,15 +2,14 @@ package spark
 
 import java.io._
 import java.util.concurrent.atomic.AtomicInteger
-import java.net.{URI, URLClassLoader}
+import java.net.URI
 
 import scala.collection.Map
 import scala.collection.generic.Growable
-import scala.collection.mutable.{ArrayBuffer, HashMap}
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
 
-import akka.actor.Actor
-import akka.actor.Actor._
-import org.apache.hadoop.fs.{FileUtil, Path}
+import org.apache.hadoop.fs.Path
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.InputFormat
 import org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -30,89 +29,83 @@ 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.mesos.{Scheduler, MesosNativeLibrary}
+import org.apache.mesos.MesosNativeLibrary
 
-import spark.broadcast._
 import spark.deploy.LocalSparkCluster
 import spark.partial.ApproximateEvaluator
 import spark.partial.PartialResult
-import spark.rdd.HadoopRDD
-import spark.rdd.NewHadoopRDD
-import spark.rdd.UnionRDD
-import spark.scheduler.ShuffleMapTask
-import spark.scheduler.DAGScheduler
-import spark.scheduler.TaskScheduler
+import spark.rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
+import spark.scheduler._
 import spark.scheduler.local.LocalScheduler
 import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
 import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
+import spark.storage.BlockManagerUI
+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
  * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
  *
  * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param jobName A name for your job, to display on the cluster web UI.
+ * @param appName A name for your application, to display on the cluster web UI.
  * @param sparkHome Location where Spark is installed on cluster nodes.
  * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
  *             system or HDFS, HTTP, HTTPS, or FTP URLs.
  * @param environment Environment variables to set on worker nodes.
  */
 class SparkContext(
-    master: String,
-    jobName: String,
-    val sparkHome: String,
-    jars: Seq[String],
-    environment: Map[String, String])
+    val master: String,
+    val appName: String,
+    val sparkHome: String = null,
+    val jars: Seq[String] = Nil,
+    val environment: Map[String, String] = Map())
   extends Logging {
 
-  /**
-   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param jobName A name for your job, to display on the cluster web UI
-   * @param sparkHome Location where Spark is installed on cluster nodes.
-   * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
-   *             system or HDFS, HTTP, HTTPS, or FTP URLs.
-   */
-  def this(master: String, jobName: String, sparkHome: String, jars: Seq[String]) =
-    this(master, jobName, sparkHome, jars, Map())
-
-  /**
-   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param jobName A name for your job, to display on the cluster web UI
-   */
-  def this(master: String, jobName: String) = this(master, jobName, null, Nil, Map())
-
   // Ensure logging is initialized before we spawn any threads
   initLogging()
 
-  // Set Spark master host and port system properties
-  if (System.getProperty("spark.master.host") == null) {
-    System.setProperty("spark.master.host", Utils.localIpAddress)
+  // Set Spark driver host and port system properties
+  if (System.getProperty("spark.driver.host") == null) {
+    System.setProperty("spark.driver.host", Utils.localIpAddress)
   }
-  if (System.getProperty("spark.master.port") == null) {
-    System.setProperty("spark.master.port", "0")
+  if (System.getProperty("spark.driver.port") == null) {
+    System.setProperty("spark.driver.port", "0")
   }
 
   private val isLocal = (master == "local" || master.startsWith("local["))
 
   // Create the Spark execution environment (cache, map output tracker, etc)
   private[spark] val env = SparkEnv.createFromSystemProperties(
-    System.getProperty("spark.master.host"),
-    System.getProperty("spark.master.port").toInt,
+    "<driver>",
+    System.getProperty("spark.driver.host"),
+    System.getProperty("spark.driver.port").toInt,
     true,
     isLocal)
   SparkEnv.set(env)
 
+  // Start the BlockManager UI
+  private[spark] val ui = new BlockManagerUI(
+    env.actorSystem, env.blockManager.master.driverActor, this)
+  ui.start()
+
   // Used to store a URL for each static file/jar together with the file's local timestamp
   private[spark] val addedFiles = HashMap[String, Long]()
   private[spark] val addedJars = HashMap[String, Long]()
 
+  // Keeps track of all persisted RDDs
+  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(_) }
 
   // Environment variables to pass to our executors
   private[spark] val executorEnvs = HashMap[String, String]()
+  // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
   for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS",
-       "SPARK_TESTING")) {
+      "SPARK_TESTING")) {
     val value = System.getenv(key)
     if (value != null) {
       executorEnvs(key) = value
@@ -130,6 +123,8 @@ class SparkContext(
     val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
     // Regular expression for connecting to Spark deploy clusters
     val SPARK_REGEX = """(spark://.*)""".r
+    //Regular expression for connection to Mesos cluster
+    val MESOS_REGEX = """(mesos://.*)""".r
 
     master match {
       case "local" =>
@@ -143,7 +138,7 @@ class SparkContext(
 
       case SPARK_REGEX(sparkUrl) =>
         val scheduler = new ClusterScheduler(this)
-        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName)
+        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
         scheduler.initialize(backend)
         scheduler
 
@@ -162,7 +157,7 @@ class SparkContext(
         val localCluster = new LocalSparkCluster(
           numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
         val sparkUrl = localCluster.start()
-        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName)
+        val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
         scheduler.initialize(backend)
         backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
           localCluster.stop()
@@ -170,14 +165,17 @@ class SparkContext(
         scheduler
 
       case _ =>
+        if (MESOS_REGEX.findFirstIn(master).isEmpty) {
+          logWarning("Master %s does not match expected format, parsing as Mesos URL".format(master))
+        }
         MesosNativeLibrary.load()
         val scheduler = new ClusterScheduler(this)
         val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
         val masterWithoutProtocol = master.replaceFirst("^mesos://", "")  // Strip initial mesos://
         val backend = if (coarseGrained) {
-          new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName)
+          new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
         } else {
-          new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName)
+          new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
         }
         scheduler.initialize(backend)
         scheduler
@@ -186,12 +184,34 @@ class SparkContext(
   taskScheduler.start()
 
   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()
+    // 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"))
+      conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
+      conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+      conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+    }
+    // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar"
+    for (key <- System.getProperties.toMap[String, String].keys if key.startsWith("spark.hadoop.")) {
+      conf.set(key.substring("spark.hadoop.".length), System.getProperty(key))
+    }
+    val bufferSize = System.getProperty("spark.buffer.size", "65536")
+    conf.set("io.file.buffer.size", bufferSize)
+    conf
+  }
+
+  private[spark] var checkpointDir: Option[String] = None
 
   // Methods for creating RDDs
 
   /** Distribute a local Scala collection to form an RDD. */
   def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
-    new ParallelCollection[T](this, seq, numSlices)
+    new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())
   }
 
   /** Distribute a local Scala collection to form an RDD. */
@@ -199,6 +219,14 @@ class SparkContext(
     parallelize(seq, numSlices)
   }
 
+  /** Distribute a local Scala collection to form an RDD, with one or more
+    * location preferences (hostnames of Spark nodes) for each object.
+    * Create a new partition for each collection item. */
+   def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = {
+    val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
+    new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
+  }
+
   /**
    * Read a text file from HDFS, a local file system (available on all nodes), or any
    * Hadoop-supported file system URI, and return it as an RDD of Strings.
@@ -231,10 +259,8 @@ class SparkContext(
       valueClass: Class[V],
       minSplits: Int = defaultMinSplits
       ) : RDD[(K, V)] = {
-    val conf = new JobConf()
+    val conf = new JobConf(hadoopConfiguration)
     FileInputFormat.setInputPaths(conf, path)
-    val bufferSize = System.getProperty("spark.buffer.size", "65536")
-    conf.set("io.file.buffer.size", bufferSize)
     new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits)
   }
 
@@ -275,8 +301,7 @@ class SparkContext(
         path,
         fm.erasure.asInstanceOf[Class[F]],
         km.erasure.asInstanceOf[Class[K]],
-        vm.erasure.asInstanceOf[Class[V]],
-        new Configuration)
+        vm.erasure.asInstanceOf[Class[V]])
   }
 
   /**
@@ -288,7 +313,7 @@ class SparkContext(
       fClass: Class[F],
       kClass: Class[K],
       vClass: Class[V],
-      conf: Configuration): RDD[(K, V)] = {
+      conf: Configuration = hadoopConfiguration): RDD[(K, V)] = {
     val job = new NewHadoopJob(conf)
     NewFileInputFormat.addInputPath(job, new Path(path))
     val updatedConf = job.getConfiguration
@@ -300,7 +325,7 @@ class SparkContext(
    * and extra configuration options to pass to the input format.
    */
   def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]](
-      conf: Configuration,
+      conf: Configuration = hadoopConfiguration,
       fClass: Class[F],
       kClass: Class[K],
       vClass: Class[V]): RDD[(K, V)] = {
@@ -365,6 +390,13 @@ class SparkContext(
       .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes))
   }
 
+
+  protected[spark] def checkpointFile[T: ClassManifest](
+      path: String
+    ): RDD[T] = {
+    new CheckpointRDD[T](this, path)
+  }
+
   /** Build the union of a list of RDDs. */
   def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
 
@@ -376,14 +408,14 @@ class SparkContext(
 
   /**
    * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
-   * to using the `+=` method. Only the master can access the accumulator's `value`.
+   * to using the `+=` method. Only the driver can access the accumulator's `value`.
    */
   def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
     new Accumulator(initialValue, param)
 
   /**
    * Create an [[spark.Accumulable]] shared variable, to which tasks can add values with `+=`.
-   * Only the master can access the accumuable's `value`.
+   * Only the driver can access the accumuable's `value`.
    * @tparam T accumulator type
    * @tparam R type that can be added to the accumulator
    */
@@ -402,15 +434,16 @@ class SparkContext(
   }
 
   /**
-   * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for
+   * Broadcast a read-only variable to the cluster, returning a [[spark.broadcast.Broadcast]] object for
    * reading it in distributed functions. The variable will be sent to each cluster only once.
    */
   def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal)
 
   /**
-   * Add a file to be downloaded into the working directory of this Spark job on every node.
+   * Add a file to be downloaded with this Spark job on every node.
    * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
-   * filesystems), or an HTTP, HTTPS or FTP URI.
+   * filesystems), or an HTTP, HTTPS or FTP URI.  To access the file in Spark jobs,
+   * use `SparkFiles.get(path)` to find its download location.
    */
   def addFile(path: String) {
     val uri = new URI(path)
@@ -423,21 +456,44 @@ class SparkContext(
     // Fetch the file locally in case a job is executed locally.
     // Jobs that run through LocalScheduler will already fetch the required dependencies,
     // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here.
-    Utils.fetchFile(path, new File("."))
+    Utils.fetchFile(path, new File(SparkFiles.getRootDirectory))
 
     logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
   }
 
+  def addSparkListener(listener: SparkListener) {
+    dagScheduler.sparkListeners += listener
+  }
+
   /**
    * Return a map from the slave to the max memory available for caching and the remaining
    * memory available for caching.
    */
-  def getSlavesMemoryStatus: Map[String, (Long, Long)] = {
+  def getExecutorMemoryStatus: Map[String, (Long, Long)] = {
     env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
       (blockManagerId.ip + ":" + blockManagerId.port, mem)
     }
   }
 
+  /**
+   * 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] = {
+    StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this)
+  }
+
+  def getStageInfo: Map[Stage,StageInfo] = {
+    dagScheduler.stageToInfos
+  }
+
+  /**
+   * Return information about blocks stored in all of the slaves
+   */
+  def getExecutorStorageStatus : Array[StorageStatus] = {
+    env.blockManager.master.getStorageStatus
+  }
+
   /**
    * Clear the job's list of files added by `addFile` so that they do not get downloaded to
    * any new nodes.
@@ -471,17 +527,23 @@ class SparkContext(
 
   /** Shut down the SparkContext. */
   def stop() {
-    dagScheduler.stop()
-    dagScheduler = null
-    taskScheduler = null
-    // TODO: Cache.stop()?
-    env.stop()
-    // Clean up locally linked files
-    clearFiles()
-    clearJars()
-    SparkEnv.set(null)
-    ShuffleMapTask.clearCache()
-    logInfo("Successfully stopped SparkContext")
+    if (dagScheduler != null) {
+      metadataCleaner.cancel()
+      dagScheduler.stop()
+      dagScheduler = null
+      taskScheduler = null
+      // TODO: Cache.stop()?
+      env.stop()
+      // Clean up locally linked files
+      clearFiles()
+      clearJars()
+      SparkEnv.set(null)
+      ShuffleMapTask.clearCache()
+      ResultTask.clearCache()
+      logInfo("Successfully stopped SparkContext")
+    } else {
+      logInfo("SparkContext already stopped")
+    }
   }
 
   /**
@@ -502,25 +564,42 @@ class SparkContext(
   }
 
   /**
-   * Run a function on a given set of partitions in an RDD and return the results. This is the main
-   * entry point to the scheduler, by which all actions get launched. The allowLocal flag specifies
-   * whether the scheduler can run the computation on the master rather than shipping it out to the
-   * cluster, for short actions like first().
+   * Run a function on a given set of partitions in an RDD and pass the results to the given
+   * handler function. This is the main entry point for all actions in Spark. The allowLocal
+   * flag specifies whether the scheduler can run the computation on the driver rather than
+   * shipping it out to the cluster, for short actions like first().
    */
   def runJob[T, U: ClassManifest](
       rdd: RDD[T],
       func: (TaskContext, Iterator[T]) => U,
       partitions: Seq[Int],
-      allowLocal: Boolean
-      ): Array[U] = {
+      allowLocal: Boolean,
+      resultHandler: (Int, U) => Unit) {
     val callSite = Utils.getSparkCallSite
     logInfo("Starting job: " + callSite)
     val start = System.nanoTime
-    val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal)
+    val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler)
     logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
+    rdd.doCheckpoint()
     result
   }
 
+  /**
+   * Run a function on a given set of partitions in an RDD and return the results as an array. The
+   * allowLocal flag specifies whether the scheduler can run the computation on the driver rather
+   * than shipping it out to the cluster, for short actions like first().
+   */
+  def runJob[T, U: ClassManifest](
+      rdd: RDD[T],
+      func: (TaskContext, Iterator[T]) => U,
+      partitions: Seq[Int],
+      allowLocal: Boolean
+      ): Array[U] = {
+    val results = new Array[U](partitions.size)
+    runJob[T, U](rdd, func, partitions, allowLocal, (index, res) => results(index) = res)
+    results
+  }
+
   /**
    * Run a job on a given set of partitions of an RDD, but take a function of type
    * `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`.
@@ -538,14 +617,37 @@ class SparkContext(
    * Run a job on all partitions in an RDD and return the results in an array.
    */
   def runJob[T, U: ClassManifest](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
-    runJob(rdd, func, 0 until rdd.splits.size, false)
+    runJob(rdd, func, 0 until rdd.partitions.size, false)
   }
 
   /**
    * Run a job on all partitions in an RDD and return the results in an array.
    */
   def runJob[T, U: ClassManifest](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
-    runJob(rdd, func, 0 until rdd.splits.size, false)
+    runJob(rdd, func, 0 until rdd.partitions.size, false)
+  }
+
+  /**
+   * Run a job on all partitions in an RDD and pass the results to a handler function.
+   */
+  def runJob[T, U: ClassManifest](
+    rdd: RDD[T],
+    processPartition: (TaskContext, Iterator[T]) => U,
+    resultHandler: (Int, U) => Unit)
+  {
+    runJob[T, U](rdd, processPartition, 0 until rdd.partitions.size, false, resultHandler)
+  }
+
+  /**
+   * Run a job on all partitions in an RDD and pass the results to a handler function.
+   */
+  def runJob[T, U: ClassManifest](
+      rdd: RDD[T],
+      processPartition: Iterator[T] => U,
+      resultHandler: (Int, U) => Unit)
+  {
+    val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter)
+    runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler)
   }
 
   /**
@@ -574,10 +676,30 @@ class SparkContext(
     return f
   }
 
-  /** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */
+  /**
+   * Set the directory under which RDDs are going to be checkpointed. The directory must
+   * be a HDFS path if running on a cluster. If the directory does not exist, it will
+   * be created. If the directory exists and useExisting is set to true, then the
+   * exisiting directory will be used. Otherwise an exception will be thrown to
+   * prevent accidental overriding of checkpoint files in the existing directory.
+   */
+  def setCheckpointDir(dir: String, useExisting: Boolean = false) {
+    val path = new Path(dir)
+    val fs = path.getFileSystem(new Configuration())
+    if (!useExisting) {
+      if (fs.exists(path)) {
+        throw new Exception("Checkpoint directory '" + path + "' already exists.")
+      } else {
+        fs.mkdirs(path)
+      }
+    }
+    checkpointDir = Some(dir)
+  }
+
+  /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
   def defaultParallelism: Int = taskScheduler.defaultParallelism
 
-  /** Default min number of splits for Hadoop RDDs when not given by user */
+  /** Default min number of partitions for Hadoop RDDs when not given by user */
   def defaultMinSplits: Int = math.min(defaultParallelism, 2)
 
   private var nextShuffleId = new AtomicInteger(0)
@@ -588,6 +710,11 @@ class SparkContext(
 
   /** Register a new RDD, returning its RDD ID */
   private[spark] def newRddId(): Int = nextRddId.getAndIncrement()
+
+  /** Called by MetadataCleaner to clean up the persistentRdds map periodically */
+  private[spark] def cleanup(cleanupTime: Long) {
+    persistentRdds.clearOldValues(cleanupTime)
+  }
 }
 
 /**
@@ -595,6 +722,7 @@ class SparkContext(
  * various Spark features.
  */
 object SparkContext {
+
   implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] {
     def addInPlace(t1: Double, t2: Double): Double = t1 + t2
     def zero(initialValue: Double) = 0.0
@@ -605,6 +733,16 @@ object SparkContext {
     def zero(initialValue: Int) = 0
   }
 
+  implicit object LongAccumulatorParam extends AccumulatorParam[Long] {
+    def addInPlace(t1: Long, t2: Long) = t1 + t2
+    def zero(initialValue: Long) = 0l
+  }
+
+  implicit object FloatAccumulatorParam extends AccumulatorParam[Float] {
+    def addInPlace(t1: Float, t2: Float) = t1 + t2
+    def zero(initialValue: Float) = 0f
+  }
+
   // TODO: Add AccumulatorParams for other types, e.g. lists and strings
 
   implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala
index 41441720a7c8f81004fb897763a6d8ae2d29e5d9..7157fd26883d3a3f7b29fb71fc272886a92ecfd5 100644
--- a/core/src/main/scala/spark/SparkEnv.scala
+++ b/core/src/main/scala/spark/SparkEnv.scala
@@ -1,7 +1,6 @@
 package spark
 
-import akka.actor.ActorSystem
-import akka.actor.ActorSystemImpl
+import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
 import akka.remote.RemoteActorRefProvider
 
 import serializer.Serializer
@@ -19,27 +18,23 @@ import spark.util.AkkaUtils
  * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set.
  */
 class SparkEnv (
+    val executorId: String,
     val actorSystem: ActorSystem,
     val serializer: Serializer,
     val closureSerializer: Serializer,
-    val cacheTracker: CacheTracker,
+    val cacheManager: CacheManager,
     val mapOutputTracker: MapOutputTracker,
     val shuffleFetcher: ShuffleFetcher,
     val broadcastManager: BroadcastManager,
     val blockManager: BlockManager,
     val connectionManager: ConnectionManager,
-    val httpFileServer: HttpFileServer
+    val httpFileServer: HttpFileServer,
+    val sparkFilesDir: String
   ) {
 
-  /** No-parameter constructor for unit tests. */
-  def this() = {
-    this(null, new JavaSerializer, new JavaSerializer, null, null, null, null, null, null, null)
-  }
-
   def stop() {
     httpFileServer.stop()
     mapOutputTracker.stop()
-    cacheTracker.stop()
     shuffleFetcher.stop()
     broadcastManager.stop()
     blockManager.stop()
@@ -63,17 +58,18 @@ object SparkEnv extends Logging {
   }
 
   def createFromSystemProperties(
+      executorId: String,
       hostname: String,
       port: Int,
-      isMaster: Boolean,
-      isLocal: Boolean
-    ) : SparkEnv = {
+      isDriver: Boolean,
+      isLocal: Boolean): SparkEnv = {
+
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port)
 
-    // Bit of a hack: If this is the master and our port was 0 (meaning bind to any free port),
-    // figure out which port number Akka actually bound to and set spark.master.port to it.
-    if (isMaster && port == 0) {
-      System.setProperty("spark.master.port", boundPort.toString)
+    // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port),
+    // figure out which port number Akka actually bound to and set spark.driver.port to it.
+    if (isDriver && port == 0) {
+      System.setProperty("spark.driver.port", boundPort.toString)
     }
 
     val classLoader = Thread.currentThread.getContextClassLoader
@@ -86,24 +82,40 @@ object SparkEnv extends Logging {
     }
 
     val serializer = instantiateClass[Serializer]("spark.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 driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt
+        val url = "akka://spark@%s:%s/user/%s".format(driverIp, driverPort, name)
+        logInfo("Connecting to " + name + ": " + url)
+        actorSystem.actorFor(url)
+      }
+    }
 
-    val masterIp: String = System.getProperty("spark.master.host", "localhost")
-    val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt
-    val blockManagerMaster = new BlockManagerMaster(
-      actorSystem, isMaster, isLocal, masterIp, masterPort)
-    val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer)
+    val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
+      "BlockManagerMaster",
+      new spark.storage.BlockManagerMasterActor(isLocal)))
+    val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer)
 
     val connectionManager = blockManager.connectionManager
 
-    val broadcastManager = new BroadcastManager(isMaster)
+    val broadcastManager = new BroadcastManager(isDriver)
 
     val closureSerializer = instantiateClass[Serializer](
       "spark.closure.serializer", "spark.JavaSerializer")
 
-    val cacheTracker = new CacheTracker(actorSystem, isMaster, blockManager)
-    blockManager.cacheTracker = cacheTracker
+    val cacheManager = new CacheManager(blockManager)
 
-    val mapOutputTracker = new MapOutputTracker(actorSystem, isMaster)
+    // Have to assign trackerActor after initialization as MapOutputTrackerActor
+    // requires the MapOutputTracker itself
+    val mapOutputTracker = new MapOutputTracker()
+    mapOutputTracker.trackerActor = registerOrLookup(
+      "MapOutputTracker",
+      new MapOutputTrackerActor(mapOutputTracker))
 
     val shuffleFetcher = instantiateClass[ShuffleFetcher](
       "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher")
@@ -112,6 +124,15 @@ object SparkEnv extends Logging {
     httpFileServer.initialize()
     System.setProperty("spark.fileserver.uri", httpFileServer.serverUri)
 
+    // Set the sparkFiles directory, used when downloading dependencies.  In local mode,
+    // this is a temporary directory; in distributed mode, this is the executor's current working
+    // directory.
+    val sparkFilesDir: String = if (isDriver) {
+      Utils.createTempDir().getAbsolutePath
+    } else {
+      "."
+    }
+
     // Warn about deprecated spark.cache.class property
     if (System.getProperty("spark.cache.class") != null) {
       logWarning("The spark.cache.class property is no longer being used! Specify storage " +
@@ -119,15 +140,18 @@ object SparkEnv extends Logging {
     }
 
     new SparkEnv(
+      executorId,
       actorSystem,
       serializer,
       closureSerializer,
-      cacheTracker,
+      cacheManager,
       mapOutputTracker,
       shuffleFetcher,
       broadcastManager,
       blockManager,
       connectionManager,
-      httpFileServer)
+      httpFileServer,
+      sparkFilesDir)
   }
+  
 }
diff --git a/core/src/main/scala/spark/SparkFiles.java b/core/src/main/scala/spark/SparkFiles.java
new file mode 100644
index 0000000000000000000000000000000000000000..566aec622c096076f3d91dbd433297d9f2c53a8e
--- /dev/null
+++ b/core/src/main/scala/spark/SparkFiles.java
@@ -0,0 +1,25 @@
+package spark;
+
+import java.io.File;
+
+/**
+ * Resolves paths to files added through `SparkContext.addFile()`.
+ */
+public class SparkFiles {
+
+  private SparkFiles() {}
+
+  /**
+   * Get the absolute path of a file added through `SparkContext.addFile()`.
+   */
+  public static String get(String filename) {
+    return new File(getRootDirectory(), filename).getAbsolutePath();
+  }
+
+  /**
+   * Get the root directory that contains files added through `SparkContext.addFile()`.
+   */
+  public static String getRootDirectory() {
+    return SparkEnv.get().sparkFilesDir();
+  }
+}
diff --git a/core/src/main/scala/spark/TaskContext.scala b/core/src/main/scala/spark/TaskContext.scala
index d2746b26b3ec1ff5f4766b2e53837c9a3be92423..dd0609026ace36a9bd616c0e783ea672ebdcc78d 100644
--- a/core/src/main/scala/spark/TaskContext.scala
+++ b/core/src/main/scala/spark/TaskContext.scala
@@ -1,12 +1,16 @@
 package spark
 
+import executor.TaskMetrics
 import scala.collection.mutable.ArrayBuffer
 
+class TaskContext(
+  val stageId: Int,
+  val splitId: Int,
+  val attemptId: Long,
+  val taskMetrics: TaskMetrics = TaskMetrics.empty()
+) extends Serializable {
 
-class TaskContext(val stageId: Int, val splitId: Int, val attemptId: Long) extends Serializable {
-
-  @transient
-  val onCompleteCallbacks = new ArrayBuffer[() => Unit]
+  @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit]
 
   // Add a callback function to be executed on task completion. An example use
   // is for HadoopRDD to register a callback to close the input stream.
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index aeed5d2f32a70f75b3984e5e57074750024ef693..81daacf958b5a03d3135f4587f2d6e411b0c6a3c 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -1,7 +1,7 @@
 package spark
 
 import java.io._
-import java.net.{NetworkInterface, InetAddress, URL, URI}
+import java.net._
 import java.util.{Locale, Random, UUID}
 import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
 import org.apache.hadoop.conf.Configuration
@@ -10,6 +10,9 @@ import scala.collection.mutable.ArrayBuffer
 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
 
 /**
  * Various utility methods used by Spark.
@@ -111,20 +114,6 @@ private object Utils extends Logging {
     }
   }
 
-  /** Copy a file on the local file system */
-  def copyFile(source: File, dest: File) {
-    val in = new FileInputStream(source)
-    val out = new FileOutputStream(dest)
-    copyStream(in, out, true)
-  }
-
-  /** Download a file from a given URL to the local filesystem */
-  def downloadFile(url: URL, localPath: String) {
-    val in = url.openStream()
-    val out = new FileOutputStream(localPath)
-    Utils.copyStream(in, out, true)
-  }
-
   /**
    * Download a file requested by the executor. Supports fetching the file in a variety of ways,
    * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter.
@@ -201,7 +190,7 @@ private object Utils extends Logging {
       Utils.execute(Seq("tar", "-xf", filename), targetDir)
     }
     // Make the file executable - That's necessary for scripts
-    FileUtil.chmod(filename, "a+x")
+    FileUtil.chmod(targetFile.getAbsolutePath, "a+x")
   }
 
   /**
@@ -251,7 +240,8 @@ private object Utils extends Logging {
         // Address resolves to something like 127.0.1.1, which happens on Debian; try to find
         // a better address using the local network interfaces
         for (ni <- NetworkInterface.getNetworkInterfaces) {
-          for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && !addr.isLoopbackAddress) {
+          for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress &&
+               !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) {
             // We've found an address that looks reasonable!
             logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" +
               " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress +
@@ -286,48 +276,28 @@ private object Utils extends Logging {
     customHostname.getOrElse(InetAddress.getLocalHost.getHostName)
   }
 
-  /**
-   * Returns a standard ThreadFactory except all threads are daemons.
-   */
-  private def newDaemonThreadFactory: ThreadFactory = {
-    new ThreadFactory {
-      def newThread(r: Runnable): Thread = {
-        var t = Executors.defaultThreadFactory.newThread (r)
-        t.setDaemon (true)
-        return t
-      }
-    }
-  }
+  private[spark] val daemonThreadFactory: ThreadFactory =
+    new ThreadFactoryBuilder().setDaemon(true).build()
 
   /**
    * Wrapper over newCachedThreadPool.
    */
-  def newDaemonCachedThreadPool(): ThreadPoolExecutor = {
-    var threadPool = Executors.newCachedThreadPool.asInstanceOf[ThreadPoolExecutor]
-
-    threadPool.setThreadFactory (newDaemonThreadFactory)
-
-    return threadPool
-  }
+  def newDaemonCachedThreadPool(): ThreadPoolExecutor =
+    Executors.newCachedThreadPool(daemonThreadFactory).asInstanceOf[ThreadPoolExecutor]
 
   /**
    * Return the string to tell how long has passed in seconds. The passing parameter should be in
    * millisecond.
    */
   def getUsedTimeMs(startTimeMs: Long): String = {
-    return " " + (System.currentTimeMillis - startTimeMs) + " ms "
+    return " " + (System.currentTimeMillis - startTimeMs) + " ms"
   }
 
   /**
    * Wrapper over newFixedThreadPool.
    */
-  def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor = {
-    var threadPool = Executors.newFixedThreadPool(nThreads).asInstanceOf[ThreadPoolExecutor]
-
-    threadPool.setThreadFactory(newDaemonThreadFactory)
-
-    return threadPool
-  }
+  def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor =
+    Executors.newFixedThreadPool(nThreads, daemonThreadFactory).asInstanceOf[ThreadPoolExecutor]
 
   /**
    * Delete a file or directory and its contents recursively.
@@ -463,4 +433,46 @@ private object Utils extends Logging {
     }
     "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine)
   }
+
+  /**
+   * Try to find a free port to bind to on the local host. This should ideally never be needed,
+   * except that, unfortunately, some of the networking libraries we currently rely on (e.g. Spray)
+   * don't let users bind to port 0 and then figure out which free port they actually bound to.
+   * We work around this by binding a ServerSocket and immediately unbinding it. This is *not*
+   * necessarily guaranteed to work, but it's the best we can do.
+   */
+  def findFreePort(): Int = {
+    val socket = new ServerSocket(0)
+    val portBound = socket.getLocalPort
+    socket.close()
+    portBound
+  }
+
+  /**
+   * Clone an object using a Spark serializer.
+   */
+  def clone[T](value: T, serializer: SerializerInstance): T = {
+    serializer.deserialize[T](serializer.serialize(value))
+  }
+
+  /**
+   * Detect whether this thread might be executing a shutdown hook. Will always return true if
+   * the current thread is a running a shutdown hook but may spuriously return true otherwise (e.g.
+   * if System.exit was just called by a concurrent thread).
+   *
+   * Currently, this detects whether the JVM is shutting down by Runtime#addShutdownHook throwing
+   * an IllegalStateException.
+   */
+  def inShutdown(): Boolean = {
+    try {
+      val hook = new Thread {
+        override def run() {}
+      }
+      Runtime.getRuntime.addShutdownHook(hook)
+      Runtime.getRuntime.removeShutdownHook(hook)
+    } catch {
+      case ise: IllegalStateException => return true
+    }
+    return false
+  }
 }
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
index 843e1bd18bdbf766819108b9abcc5eec09afa3f5..16692c044037c236f5c8c9469183837826df9f23 100644
--- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
@@ -6,8 +6,8 @@ import spark.api.java.function.{Function => JFunction}
 import spark.util.StatCounter
 import spark.partial.{BoundedDouble, PartialResult}
 import spark.storage.StorageLevel
-
 import java.lang.Double
+import spark.Partitioner
 
 class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
 
@@ -44,7 +44,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
   /**
    * Return a new RDD containing the distinct elements in this RDD.
    */
-  def distinct(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numSplits))
+  def distinct(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numPartitions))
 
   /**
    * Return a new RDD containing only the elements that satisfy a predicate.
@@ -52,6 +52,38 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
   def filter(f: JFunction[Double, java.lang.Boolean]): JavaDoubleRDD =
     fromRDD(srdd.filter(x => f(x).booleanValue()))
 
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  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`.
+   * 
+   * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
+   * RDD will be <= us.
+   */
+  def subtract(other: JavaDoubleRDD): JavaDoubleRDD =
+    fromRDD(srdd.subtract(other))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  def subtract(other: JavaDoubleRDD, numPartitions: Int): JavaDoubleRDD =
+    fromRDD(srdd.subtract(other, numPartitions))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  def subtract(other: JavaDoubleRDD, p: Partitioner): JavaDoubleRDD =
+    fromRDD(srdd.subtract(other, p))
+
   /**
    * Return a sampled subset of this RDD.
    */
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
index 8ce32e0e2fd21b57ef28f9c8c044ef2727be64e8..30084df4e20fc99e5ed00a69f76d661b8047b61d 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
@@ -19,6 +19,7 @@ import spark.OrderedRDDFunctions
 import spark.storage.StorageLevel
 import spark.HashPartitioner
 import spark.Partitioner
+import spark.Partitioner._
 import spark.RDD
 import spark.SparkContext.rddToPairRDDFunctions
 
@@ -54,14 +55,25 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   /**
    * Return a new RDD containing the distinct elements in this RDD.
    */
-  def distinct(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numSplits))
+  def distinct(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numPartitions))
 
   /**
    * Return a new RDD containing only the elements that satisfy a predicate.
    */
-  def filter(f: Function[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] =
+  def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] =
     new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue()))
 
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  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.
    */
@@ -97,7 +109,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * In addition, users can control the partitioning of the output RDD, and whether to perform
    * map-side aggregation (if a mapper can produce multiple items with the same key).
    */
-  def combineByKey[C](createCombiner: Function[V, C],
+  def combineByKey[C](createCombiner: JFunction[V, C],
     mergeValue: JFunction2[C, V, C],
     mergeCombiners: JFunction2[C, C, C],
     partitioner: Partitioner): JavaPairRDD[K, C] = {
@@ -117,8 +129,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   def combineByKey[C](createCombiner: JFunction[V, C],
     mergeValue: JFunction2[C, V, C],
     mergeCombiners: JFunction2[C, C, C],
-    numSplits: Int): JavaPairRDD[K, C] =
-    combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits))
+    numPartitions: Int): JavaPairRDD[K, C] =
+    combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions))
 
   /**
    * Merge the values for each key using an associative reduce function. This will also perform
@@ -154,13 +166,37 @@ 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
-   * "combiner" in MapReduce. Output will be hash-partitioned with numSplits splits.
+   * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
    */
-  def reduceByKey(func: JFunction2[V, V, V], numSplits: Int): JavaPairRDD[K, V] =
-    fromRDD(rdd.reduceByKey(func, numSplits))
+  def reduceByKey(func: JFunction2[V, V, V], numPartitions: Int): JavaPairRDD[K, V] =
+    fromRDD(rdd.reduceByKey(func, numPartitions))
 
   /**
    * Group the values for each key in the RDD into a single sequence. Allows controlling the
@@ -171,10 +207,31 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
 
   /**
    * Group the values for each key in the RDD into a single sequence. Hash-partitions the
-   * resulting RDD with into `numSplits` partitions.
+   * resulting RDD with into `numPartitions` partitions.
+   */
+  def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] =
+    fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions)))
+
+  /**
+   * 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.
+   */
+  def subtract(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] =
+    fromRDD(rdd.subtract(other))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
    */
-  def groupByKey(numSplits: Int): JavaPairRDD[K, JList[V]] =
-    fromRDD(groupByResultToJava(rdd.groupByKey(numSplits)))
+  def subtract(other: JavaPairRDD[K, V], numPartitions: Int): JavaPairRDD[K, V] =
+    fromRDD(rdd.subtract(other, numPartitions))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  def subtract(other: JavaPairRDD[K, V], p: Partitioner): JavaPairRDD[K, V] =
+    fromRDD(rdd.subtract(other, p))
 
   /**
    * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine`
@@ -215,30 +272,30 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
     fromRDD(rdd.rightOuterJoin(other, partitioner))
 
   /** 
-   * Simplified version of combineByKey that hash-partitions the resulting RDD using the default
-   * parallelism level.
+   * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing
+   * partitioner/parallelism level.
    */
   def combineByKey[C](createCombiner: JFunction[V, C],
     mergeValue: JFunction2[C, V, C],
     mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = {
     implicit val cm: ClassManifest[C] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
-    fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners))
+    fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd)))
   }
 
   /**
    * 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
-   * "combiner" in MapReduce. Output will be hash-partitioned with the default parallelism level.
+   * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/
+   * parallelism level.
    */
   def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = {
-    val partitioner = rdd.defaultPartitioner(rdd)
-    fromRDD(reduceByKey(partitioner, func))
+    fromRDD(reduceByKey(defaultPartitioner(rdd), func))
   }
 
   /**
    * Group the values for each key in the RDD into a single sequence. Hash-partitions the
-   * resulting RDD with the default parallelism level.
+   * resulting RDD with the existing partitioner/parallelism level.
    */
   def groupByKey(): JavaPairRDD[K, JList[V]] =
     fromRDD(groupByResultToJava(rdd.groupByKey()))
@@ -256,14 +313,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
    * (k, v2) is in `other`. Performs a hash join across the cluster.
    */
-  def join[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, W)] =
-    fromRDD(rdd.join(other, numSplits))
+  def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] =
+    fromRDD(rdd.join(other, numPartitions))
 
   /**
    * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
    * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
    * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
-   * using the default level of parallelism.
+   * using the existing partitioner/parallelism level.
    */
   def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Option[W])] =
     fromRDD(rdd.leftOuterJoin(other))
@@ -272,16 +329,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
    * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
    * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
-   * into `numSplits` partitions.
+   * into `numPartitions` partitions.
    */
-  def leftOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, Option[W])] =
-    fromRDD(rdd.leftOuterJoin(other, numSplits))
+  def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Option[W])] =
+    fromRDD(rdd.leftOuterJoin(other, numPartitions))
 
   /**
    * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
    * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
    * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
-   * RDD using the default parallelism level.
+   * RDD using the existing partitioner/parallelism level.
    */
   def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Option[V], W)] =
     fromRDD(rdd.rightOuterJoin(other))
@@ -292,8 +349,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
    * RDD into the given number of partitions.
    */
-  def rightOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (Option[V], W)] =
-    fromRDD(rdd.rightOuterJoin(other, numSplits))
+  def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Option[V], W)] =
+    fromRDD(rdd.rightOuterJoin(other, numPartitions))
 
   /**
    * Return the key-value pairs in this RDD to the master as a Map.
@@ -304,7 +361,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * Pass each value in the key-value pair RDD through a map function without changing the keys;
    * this also retains the original RDD's partitioning.
    */
-  def mapValues[U](f: Function[V, U]): JavaPairRDD[K, U] = {
+  def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = {
     implicit val cm: ClassManifest[U] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
     fromRDD(rdd.mapValues(f))
@@ -357,16 +414,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
    * list of values for that key in `this` as well as `other`.
    */
-  def cogroup[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (JList[V], JList[W])]
-  = fromRDD(cogroupResultToJava(rdd.cogroup(other, numSplits)))
+  def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])]
+  = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions)))
 
   /**
    * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
    * tuple with the list of values for that key in `this`, `other1` and `other2`.
    */
-  def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numSplits: Int)
+  def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int)
   : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] =
-    fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numSplits)))
+    fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions)))
 
   /** Alias for cogroup. */
   def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] =
@@ -447,7 +504,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    */
   def sortByKey(ascending: Boolean): JavaPairRDD[K, V] = {
     val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]]
-    sortByKey(comp, true)
+    sortByKey(comp, ascending)
   }
 
   /**
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala
index ac31350ec3374e6a6396087b2c37681ecfe79f81..e29f1e58990b8ec7e50d6efcf96fe1cc858bb5ea 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDD.scala
@@ -30,7 +30,7 @@ JavaRDDLike[T, JavaRDD[T]] {
   /**
    * Return a new RDD containing the distinct elements in this RDD.
    */
-  def distinct(numSplits: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numSplits))
+  def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions))
   
   /**
    * Return a new RDD containing only the elements that satisfy a predicate.
@@ -38,6 +38,17 @@ JavaRDDLike[T, JavaRDD[T]] {
   def filter(f: JFunction[T, java.lang.Boolean]): JavaRDD[T] =
     wrapRDD(rdd.filter((x => f(x).booleanValue())))
 
+  /**
+   * Return a new RDD that is reduced into `numPartitions` partitions.
+   */
+  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.
    */
@@ -50,6 +61,26 @@ JavaRDDLike[T, JavaRDD[T]] {
    */
   def union(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.union(other.rdd))
 
+  /**
+   * 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.
+   */
+  def subtract(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.subtract(other))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  def subtract(other: JavaRDD[T], numPartitions: Int): JavaRDD[T] =
+    wrapRDD(rdd.subtract(other, numPartitions))
+
+  /**
+   * Return an RDD with the elements from `this` that are not in `other`.
+   */
+  def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] =
+    wrapRDD(rdd.subtract(other, p))
+
 }
 
 object JavaRDD {
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
index d15f6dd02f5acfe30827355a2d23e52bcc8adab8..d884529d7a6f552227deb3989912efeff13cd5f2 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -4,11 +4,12 @@ import java.util.{List => JList}
 import scala.Tuple2
 import scala.collection.JavaConversions._
 
-import spark.{SparkContext, Split, RDD, TaskContext}
+import spark.{SparkContext, Partition, RDD, TaskContext}
 import spark.api.java.JavaPairRDD._
 import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
 import spark.partial.{PartialResult, BoundedDouble}
 import spark.storage.StorageLevel
+import com.google.common.base.Optional
 
 
 trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
@@ -19,7 +20,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
   def rdd: RDD[T]
 
   /** Set of partitions in this RDD. */
-  def splits: JList[Split] = new java.util.ArrayList(rdd.splits.toSeq)
+  def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq)
 
   /** The [[spark.SparkContext]] that this RDD was created on. */
   def context: SparkContext = rdd.context
@@ -35,7 +36,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
    * This should ''not'' be called by users directly, but is available for implementors of custom
    * subclasses of RDD.
    */
-  def iterator(split: Split, taskContext: TaskContext): java.util.Iterator[T] =
+  def iterator(split: Partition, taskContext: TaskContext): java.util.Iterator[T] =
     asJavaIterator(rdd.iterator(split, taskContext))
 
   // Transformations (return a new RDD)
@@ -84,10 +85,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
    *  Return a new RDD by first applying a function to all elements of this
    *  RDD, and then flattening the results.
    */
-  def flatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairRDD[K, V] = {
+  def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = {
     import scala.collection.JavaConverters._
     def fn = (x: T) => f.apply(x).asScala
-    def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+    def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
     JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType())
   }
 
@@ -110,8 +111,8 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
   /**
    * Return a new RDD by applying a function to each partition of this RDD.
    */
-  def mapPartitions[K, V](f: PairFlatMapFunction[java.util.Iterator[T], K, V]):
-  JavaPairRDD[K, V] = {
+  def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]):
+  JavaPairRDD[K2, V2] = {
     def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
     JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(f.keyType(), f.valueType())
   }
@@ -146,12 +147,12 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
    * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
    * mapping to that key.
    */
-  def groupBy[K](f: JFunction[T, K], numSplits: Int): JavaPairRDD[K, JList[T]] = {
+  def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = {
     implicit val kcm: ClassManifest[K] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
     implicit val vcm: ClassManifest[JList[T]] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]]
-    JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numSplits)(f.returnType)))(kcm, vcm)
+    JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm)
   }
 
   /**
@@ -201,7 +202,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
   }
 
   /**
-   * Reduces the elements of this RDD using the specified associative binary operator.
+   * Reduces the elements of this RDD using the specified commutative and associative binary operator.
    */
   def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f)
 
@@ -306,4 +307,33 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
     implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
     JavaPairRDD.fromRDD(rdd.keyBy(f))
   }
+
+  /**
+   * Mark this RDD for checkpointing. It will be saved to a file inside the checkpoint
+   * directory set with SparkContext.setCheckpointDir() and all references to its parent
+   * RDDs will be removed. This function must be called before any job has been
+   * executed on this RDD. It is strongly recommended that this RDD is persisted in
+   * memory, otherwise saving it on a file will require recomputation.
+   */
+  def checkpoint() = rdd.checkpoint()
+
+  /**
+   * Return whether this RDD has been checkpointed or not
+   */
+  def isCheckpointed: Boolean = rdd.isCheckpointed
+
+  /**
+   * Gets the name of the file to which this RDD was checkpointed
+   */
+  def getCheckpointFile(): Optional[String] = {
+    rdd.getCheckpointFile match {
+      case Some(file) => Optional.of(file)
+      case _ => Optional.absent()
+    }
+  }
+
+  /** A description of this RDD and its recursive dependencies for debugging. */
+  def toDebugString(): String = {
+    rdd.toDebugString
+  }
 }
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index 88ab2846bebc082467ee9234bd9a2dc62ed962dc..5f18b1e15bd69c866e0e52b19f7b8b1e2303c13d 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -23,41 +23,41 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
 
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param jobName A name for your job, to display on the cluster web UI
+   * @param appName A name for your application, to display on the cluster web UI
    */
-  def this(master: String, jobName: String) = this(new SparkContext(master, jobName))
+  def this(master: String, appName: String) = this(new SparkContext(master, appName))
 
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param jobName A name for your job, to display on the cluster web UI
+   * @param appName A name for your application, to display on the cluster web UI
    * @param sparkHome The SPARK_HOME directory on the slave nodes
-   * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
-   *             system or HDFS, HTTP, HTTPS, or FTP URLs.
+   * @param jarFile JAR file to send to the cluster. This can be a path on the local file system
+   *                or an HDFS, HTTP, HTTPS, or FTP URL.
    */
-  def this(master: String, jobName: String, sparkHome: String, jarFile: String) =
-    this(new SparkContext(master, jobName, sparkHome, Seq(jarFile)))
+  def this(master: String, appName: String, sparkHome: String, jarFile: String) =
+    this(new SparkContext(master, appName, sparkHome, Seq(jarFile)))
 
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param jobName A name for your job, to display on the cluster web UI
+   * @param appName A name for your application, to display on the cluster web UI
    * @param sparkHome The SPARK_HOME directory on the slave nodes
    * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
    *             system or HDFS, HTTP, HTTPS, or FTP URLs.
    */
-  def this(master: String, jobName: String, sparkHome: String, jars: Array[String]) =
-    this(new SparkContext(master, jobName, sparkHome, jars.toSeq))
+  def this(master: String, appName: String, sparkHome: String, jars: Array[String]) =
+    this(new SparkContext(master, appName, sparkHome, jars.toSeq))
 
   /**
    * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
-   * @param jobName A name for your job, to display on the cluster web UI
+   * @param appName A name for your application, to display on the cluster web UI
    * @param sparkHome The SPARK_HOME directory on the slave nodes
    * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
    *             system or HDFS, HTTP, HTTPS, or FTP URLs.
    * @param environment Environment variables to set on worker nodes
    */
-  def this(master: String, jobName: String, sparkHome: String, jars: Array[String],
+  def this(master: String, appName: String, sparkHome: String, jars: Array[String],
       environment: JMap[String, String]) =
-    this(new SparkContext(master, jobName, sparkHome, jars.toSeq, environment))
+    this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment))
 
   private[spark] val env = sc.env
 
@@ -323,9 +323,10 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
   def getSparkHome(): Option[String] = sc.getSparkHome()
 
   /**
-   * Add a file to be downloaded into the working directory of this Spark job on every node.
+   * Add a file to be downloaded with this Spark job on every node.
    * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
-   * filesystems), or an HTTP, HTTPS or FTP URI.
+   * filesystems), or an HTTP, HTTPS or FTP URI.  To access the file in Spark jobs,
+   * use `SparkFiles.get(path)` to find its download location.
    */
   def addFile(path: String) {
     sc.addFile(path)
@@ -355,6 +356,40 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
   def clearFiles() {
     sc.clearFiles()
   }
+
+  /**
+   * Returns the Hadoop configuration used for the Hadoop code (e.g. file systems) we reuse.
+   */
+  def hadoopConfiguration(): Configuration = {
+    sc.hadoopConfiguration
+  }
+
+  /**
+   * Set the directory under which RDDs are going to be checkpointed. The directory must
+   * be a HDFS path if running on a cluster. If the directory does not exist, it will
+   * be created. If the directory exists and useExisting is set to true, then the
+   * exisiting directory will be used. Otherwise an exception will be thrown to
+   * prevent accidental overriding of checkpoint files in the existing directory.
+   */
+  def setCheckpointDir(dir: String, useExisting: Boolean) {
+    sc.setCheckpointDir(dir, useExisting)
+  }
+
+  /**
+   * Set the directory under which RDDs are going to be checkpointed. The directory must
+   * be a HDFS path if running on a cluster. If the directory does not exist, it will
+   * be created. If the directory exists, an exception will be thrown to prevent accidental
+   * overriding of checkpoint files.
+   */
+  def setCheckpointDir(dir: String) {
+    sc.setCheckpointDir(dir)
+  }
+
+  protected def checkpointFile[T](path: String): JavaRDD[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    new JavaRDD(sc.checkpointFile(path))
+  }
 }
 
 object JavaSparkContext {
diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java
index 722af3c06c668322b9be91fe5d07325e79f5779b..5e5845ac3a65b62c65b7aaaac9720bff73eb646f 100644
--- a/core/src/main/scala/spark/api/java/StorageLevels.java
+++ b/core/src/main/scala/spark/api/java/StorageLevels.java
@@ -17,4 +17,15 @@ public class StorageLevels {
   public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2);
   public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, 1);
   public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2);
+
+  /**
+   * Create a new StorageLevel object.
+   * @param useDisk saved to disk, if true
+   * @param useMemory saved to memory, if true
+   * @param deserialized saved as deserialized objects, if true
+   * @param replication replication factor
+   */
+  public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) {
+    return StorageLevel.apply(useDisk, useMemory, deserialized, replication);
+  }
 }
diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala
index 648d9402b07951b654a06238ef0cf2763bc577aa..d618c098c2bed7f74773c3ba028319fbfcd7f070 100644
--- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala
+++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala
@@ -6,8 +6,17 @@ import java.util.Arrays
 
 /**
  * A [[spark.Partitioner]] that performs handling of byte arrays, for use by the Python API.
+ *
+ * Stores the unique id() of the Python-side partitioning function so that it is incorporated into
+ * equality comparisons.  Correctness requires that the id is a unique identifier for the
+ * lifetime of the program (i.e. that it is not re-used as the id of a different partitioning
+ * function).  This can be ensured by using the Python id() function and maintaining a reference
+ * to the Python partitioning function so that its id() is not reused.
  */
-private[spark] class PythonPartitioner(override val numPartitions: Int) extends Partitioner {
+private[spark] class PythonPartitioner(
+  override val numPartitions: Int,
+  val pyPartitionFunctionId: Long)
+  extends Partitioner {
 
   override def getPartition(key: Any): Int = {
     if (key == null) {
@@ -32,7 +41,7 @@ private[spark] class PythonPartitioner(override val numPartitions: Int) extends
 
   override def equals(other: Any): Boolean = other match {
     case h: PythonPartitioner =>
-      h.numPartitions == numPartitions
+      h.numPartitions == numPartitions && h.pyPartitionFunctionId == pyPartitionFunctionId
     case _ =>
       false
   }
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala
index f431ef28d344b4575e5f6f7a64f3cf539c77726b..9b4d54ab4e0461364d643bb857b267ed4e05bed6 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/spark/api/python/PythonRDD.scala
@@ -1,7 +1,8 @@
 package spark.api.python
 
 import java.io._
-import java.util.{List => JList}
+import java.net._
+import java.util.{List => JList, ArrayList => JArrayList, Collections}
 
 import scala.collection.JavaConversions._
 import scala.io.Source
@@ -10,33 +11,32 @@ import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
 import spark.broadcast.Broadcast
 import spark._
 import spark.rdd.PipedRDD
-import java.util
 
 
 private[spark] class PythonRDD[T: ClassManifest](
-  parent: RDD[T],
-  command: Seq[String],
-  envVars: java.util.Map[String, String],
-  preservePartitoning: Boolean,
-  pythonExec: String,
-  broadcastVars: java.util.List[Broadcast[Array[Byte]]])
-  extends RDD[Array[Byte]](parent.context) {
+    parent: RDD[T],
+    command: Seq[String],
+    envVars: java.util.Map[String, String],
+    preservePartitoning: Boolean,
+    pythonExec: String,
+    broadcastVars: JList[Broadcast[Array[Byte]]],
+    accumulator: Accumulator[JList[Array[Byte]]])
+  extends RDD[Array[Byte]](parent) {
 
   // Similar to Runtime.exec(), if we are given a single string, split it into words
   // using a standard StringTokenizer (i.e. by spaces)
   def this(parent: RDD[T], command: String, envVars: java.util.Map[String, String],
-    preservePartitoning: Boolean, pythonExec: String,
-    broadcastVars: java.util.List[Broadcast[Array[Byte]]]) =
+      preservePartitoning: Boolean, pythonExec: String,
+      broadcastVars: JList[Broadcast[Array[Byte]]],
+      accumulator: Accumulator[JList[Array[Byte]]]) =
     this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec,
-      broadcastVars)
+      broadcastVars, accumulator)
 
-  override def splits = parent.splits
-
-  override val dependencies = List(new OneToOneDependency(parent))
+  override def getPartitions = parent.partitions
 
   override val partitioner = if (preservePartitoning) parent.partitioner else None
 
-  override def compute(split: Split, context: TaskContext): Iterator[Array[Byte]] = {
+  override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
     val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME")
 
     val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/python/pyspark/worker.py"))
@@ -51,7 +51,7 @@ private[spark] class PythonRDD[T: ClassManifest](
     val env = SparkEnv.get
 
     // Start a thread to print the process's stderr to ours
-    new Thread("stderr reader for " + command) {
+    new Thread("stderr reader for " + pythonExec) {
       override def run() {
         for (line <- Source.fromInputStream(proc.getErrorStream).getLines) {
           System.err.println(line)
@@ -60,13 +60,15 @@ private[spark] class PythonRDD[T: ClassManifest](
     }.start()
 
     // Start a thread to feed the process input from our parent's iterator
-    new Thread("stdin writer for " + command) {
+    new Thread("stdin writer for " + pythonExec) {
       override def run() {
         SparkEnv.set(env)
         val out = new PrintWriter(proc.getOutputStream)
         val dOut = new DataOutputStream(proc.getOutputStream)
-        // Split index
+        // Partition index
         dOut.writeInt(split.index)
+        // sparkFilesDir
+        PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut)
         // Broadcast variables
         dOut.writeInt(broadcastVars.length)
         for (broadcast <- broadcastVars) {
@@ -93,18 +95,36 @@ private[spark] class PythonRDD[T: ClassManifest](
     // Return an iterator that read lines from the process's stdout
     val stream = new DataInputStream(proc.getInputStream)
     return new Iterator[Array[Byte]] {
-      def next() = {
+      def next(): Array[Byte] = {
         val obj = _nextObj
         _nextObj = read()
         obj
       }
 
-      private def read() = {
+      private def read(): Array[Byte] = {
         try {
-          val length = stream.readInt()
-          val obj = new Array[Byte](length)
-          stream.readFully(obj)
-          obj
+          stream.readInt() match {
+            case length if length > 0 =>
+              val obj = new Array[Byte](length)
+              stream.readFully(obj)
+              obj
+            case -2 =>
+              // Signals that an exception has been thrown in python
+              val exLength = stream.readInt()
+              val obj = new Array[Byte](exLength)
+              stream.readFully(obj)
+              throw new PythonException(new String(obj))
+            case -1 =>
+              // We've finished the data section of the output, but we can still read some
+              // accumulator updates; let's do that, breaking when we get EOFException
+              while (true) {
+                val len2 = stream.readInt()
+                val update = new Array[Byte](len2)
+                stream.readFully(update)
+                accumulator += Collections.singletonList(update)
+              }
+              new Array[Byte](0)
+          }
         } catch {
           case eof: EOFException => {
             val exitStatus = proc.waitFor()
@@ -126,15 +146,17 @@ private[spark] class PythonRDD[T: ClassManifest](
   val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
 }
 
+/** Thrown for exceptions in user Python code. */
+private class PythonException(msg: String) extends Exception(msg)
+
 /**
  * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python.
  * This is used by PySpark's shuffle operations.
  */
 private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
-  RDD[(Array[Byte], Array[Byte])](prev.context) {
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override def compute(split: Split, context: TaskContext) =
+  RDD[(Array[Byte], Array[Byte])](prev) {
+  override def getPartitions = prev.partitions
+  override def compute(split: Partition, context: TaskContext) =
     prev.iterator(split, context).grouped(2).map {
       case Seq(a, b) => (a, b)
       case x          => throw new Exception("PairwiseRDD: unexpected value: " + x)
@@ -216,6 +238,11 @@ private[spark] object PythonRDD {
   }
 
   def writeIteratorToPickleFile[T](items: java.util.Iterator[T], filename: String) {
+    import scala.collection.JavaConverters._
+    writeIteratorToPickleFile(items.asScala, filename)
+  }
+
+  def writeIteratorToPickleFile[T](items: Iterator[T], filename: String) {
     val file = new DataOutputStream(new FileOutputStream(filename))
     for (item <- items) {
       writeAsPickle(item, file)
@@ -223,8 +250,10 @@ private[spark] object PythonRDD {
     file.close()
   }
 
-  def takePartition[T](rdd: RDD[T], partition: Int): java.util.Iterator[T] =
-    rdd.context.runJob(rdd, ((x: Iterator[T]) => x), Seq(partition), true).head
+  def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = {
+    implicit val cm : ClassManifest[T] = rdd.elementClassManifest
+    rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator
+  }
 }
 
 private object Pickle {
@@ -238,11 +267,43 @@ private object Pickle {
   val APPENDS: Byte = 'e'
 }
 
-private class ExtractValue extends spark.api.java.function.Function[(Array[Byte],
-  Array[Byte]), Array[Byte]] {
-  override def call(pair: (Array[Byte], Array[Byte])) : Array[Byte] = pair._2
-}
-
 private class BytesToString extends spark.api.java.function.Function[Array[Byte], String] {
   override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8")
 }
+
+/**
+ * Internal class that acts as an `AccumulatorParam` for Python accumulators. Inside, it
+ * collects a list of pickled strings that we pass to Python through a socket.
+ */
+class PythonAccumulatorParam(@transient serverHost: String, serverPort: Int)
+  extends AccumulatorParam[JList[Array[Byte]]] {
+  
+  override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
+
+  override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]])
+      : JList[Array[Byte]] = {
+    if (serverHost == null) {
+      // This happens on the worker node, where we just want to remember all the updates
+      val1.addAll(val2)
+      val1
+    } else {
+      // This happens on the master, where we pass the updates to Python through a socket
+      val socket = new Socket(serverHost, serverPort)
+      val in = socket.getInputStream
+      val out = new DataOutputStream(socket.getOutputStream)
+      out.writeInt(val2.size)
+      for (array <- val2) {
+        out.writeInt(array.length)
+        out.write(array)
+      }
+      out.flush()
+      // Wait for a byte from the Python side as an acknowledgement
+      val byteRead = in.read()
+      if (byteRead == -1) {
+        throw new SparkException("EOF reached before Python server acknowledged")
+      }
+      socket.close()
+      null
+    }
+  }
+}
diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
index 386f505f2a9db12482c81386bac4414a9c4f4d9e..adcb2d2415eb3f03721538bbc8235f9839188884 100644
--- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
@@ -31,7 +31,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
   @transient var totalBlocks = -1
   @transient var hasBlocks = new AtomicInteger(0)
 
-  // Used ONLY by Master to track how many unique blocks have been sent out
+  // Used ONLY by driver to track how many unique blocks have been sent out
   @transient var sentBlocks = new AtomicInteger(0)
 
   @transient var listenPortLock = new Object
@@ -42,7 +42,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
 
   @transient var serveMR: ServeMultipleRequests = null
 
-  // Used only in Master
+  // Used only in driver
   @transient var guideMR: GuideMultipleRequests = null
 
   // Used only in Workers
@@ -99,14 +99,14 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
     }
 
     // Must always come AFTER listenPort is created
-    val masterSource =
+    val driverSource =
       SourceInfo(hostAddress, listenPort, totalBlocks, totalBytes)
     hasBlocksBitVector.synchronized {
-      masterSource.hasBlocksBitVector = hasBlocksBitVector
+      driverSource.hasBlocksBitVector = hasBlocksBitVector
     }
 
     // In the beginning, this is the only known source to Guide
-    listOfSources += masterSource
+    listOfSources += driverSource
 
     // Register with the Tracker
     MultiTracker.registerBroadcast(id,
@@ -122,7 +122,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
 
         case None =>
           logInfo("Started reading broadcast variable " + id)
-          // Initializing everything because Master will only send null/0 values
+          // Initializing everything because driver will only send null/0 values
           // Only the 1st worker in a node can be here. Others will get from cache
           initializeWorkerVariables()
 
@@ -151,7 +151,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
     }
   }
 
-  // Initialize variables in the worker node. Master sends everything as 0/null
+  // Initialize variables in the worker node. Driver sends everything as 0/null
   private def initializeWorkerVariables() {
     arrayOfBlocks = null
     hasBlocksBitVector = null
@@ -248,7 +248,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
       // Receive source information from Guide
       var suitableSources =
         oisGuide.readObject.asInstanceOf[ListBuffer[SourceInfo]]
-      logDebug("Received suitableSources from Master " + suitableSources)
+      logDebug("Received suitableSources from Driver " + suitableSources)
 
       addToListOfSources(suitableSources)
 
@@ -532,7 +532,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
               oosSource.writeObject(blockToAskFor)
               oosSource.flush()
 
-              // CHANGED: Master might send some other block than the one
+              // CHANGED: Driver might send some other block than the one
               // requested to ensure fast spreading of all blocks.
               val recvStartTime = System.currentTimeMillis
               val bcBlock = oisSource.readObject.asInstanceOf[BroadcastBlock]
@@ -982,9 +982,9 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
             // Receive which block to send
             var blockToSend = ois.readObject.asInstanceOf[Int]
 
-            // If it is master AND at least one copy of each block has not been
+            // If it is driver AND at least one copy of each block has not been
             // sent out already, MODIFY blockToSend
-            if (MultiTracker.isMaster && sentBlocks.get < totalBlocks) {
+            if (MultiTracker.isDriver && sentBlocks.get < totalBlocks) {
               blockToSend = sentBlocks.getAndIncrement
             }
 
@@ -1031,7 +1031,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
 
 private[spark] class BitTorrentBroadcastFactory
 extends BroadcastFactory {
-  def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) }
+  def initialize(isDriver: Boolean) { MultiTracker.initialize(isDriver) }
 
   def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
     new BitTorrentBroadcast[T](value_, isLocal, id)
diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala
index 2ffe7f741d8d089aec74af3958410a9c436bf36f..415bde5d677cb3b0acf59ecc29d86be776f519c9 100644
--- a/core/src/main/scala/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/spark/broadcast/Broadcast.scala
@@ -15,7 +15,7 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
 }
 
 private[spark] 
-class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializable {
+class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable {
 
   private var initialized = false
   private var broadcastFactory: BroadcastFactory = null
@@ -33,7 +33,7 @@ class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializabl
           Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
 
         // Initialize appropriate BroadcastFactory and BroadcastObject
-        broadcastFactory.initialize(isMaster)
+        broadcastFactory.initialize(isDriver)
 
         initialized = true
       }
@@ -49,5 +49,5 @@ class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializabl
   def newBroadcast[T](value_ : T, isLocal: Boolean) =
     broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement())
 
-  def isMaster = isMaster_
+  def isDriver = _isDriver
 }
diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
index ab6d302827d7da88a03a2427f981dd360516a103..5c6184c3c715952831aac89ba32b65738b5e83cc 100644
--- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
@@ -7,7 +7,7 @@ package spark.broadcast
  * entire Spark job.
  */
 private[spark] trait BroadcastFactory {
-  def initialize(isMaster: Boolean): Unit
-  def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long): Broadcast[T]
+  def initialize(isDriver: Boolean): Unit
+  def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T]
   def stop(): Unit
 }
diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
index 856a4683a95da04cb8a5e4a659e9b78b52a7cba0..7e30b8f7d21b23c725516dfdbce5976e4905a806 100644
--- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
@@ -11,6 +11,7 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
 
 import spark._
 import spark.storage.StorageLevel
+import util.{MetadataCleaner, TimeStampedHashSet}
 
 private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
 extends Broadcast[T](id) with Logging with Serializable {
@@ -47,7 +48,7 @@ extends Broadcast[T](id) with Logging with Serializable {
 }
 
 private[spark] class HttpBroadcastFactory extends BroadcastFactory {
-  def initialize(isMaster: Boolean) { HttpBroadcast.initialize(isMaster) }
+  def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) }
 
   def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
     new HttpBroadcast[T](value_, isLocal, id)
@@ -64,12 +65,16 @@ private object HttpBroadcast extends Logging {
   private var serverUri: String = null
   private var server: HttpServer = null
 
-  def initialize(isMaster: Boolean) {
+  private val files = new TimeStampedHashSet[String]
+  private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup)
+
+
+  def initialize(isDriver: Boolean) {
     synchronized {
       if (!initialized) {
         bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
         compress = System.getProperty("spark.broadcast.compress", "true").toBoolean
-        if (isMaster) {
+        if (isDriver) {
           createServer()
         }
         serverUri = System.getProperty("spark.httpBroadcast.uri")
@@ -85,6 +90,7 @@ private object HttpBroadcast extends Logging {
         server = null
       }
       initialized = false
+      cleaner.cancel()
     }
   }
 
@@ -108,6 +114,7 @@ private object HttpBroadcast extends Logging {
     val serOut = ser.serializeStream(out)
     serOut.writeObject(value)
     serOut.close()
+    files += file.getAbsolutePath
   }
 
   def read[T](id: Long): T = {
@@ -123,4 +130,21 @@ private object HttpBroadcast extends Logging {
     serIn.close()
     obj
   }
+
+  def cleanup(cleanupTime: Long) {
+    val iterator = files.internalMap.entrySet().iterator()
+    while(iterator.hasNext) {
+      val entry = iterator.next()
+      val (file, time) = (entry.getKey, entry.getValue)
+      if (time < cleanupTime) {
+        try {
+          iterator.remove()
+          new File(file.toString).delete()
+          logInfo("Deleted broadcast file '" + file + "'")
+        } catch {
+          case e: Exception => logWarning("Could not delete broadcast file '" + file + "'", e)
+        }
+      }
+    }
+  }
 }
diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala
index 5e76dedb942dbf1736320f656ea6ce06dacc2a0f..3fd77af73f17624543f3cca61b3f8900cae040fb 100644
--- a/core/src/main/scala/spark/broadcast/MultiTracker.scala
+++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala
@@ -23,25 +23,24 @@ extends Logging {
   var ranGen = new Random
 
   private var initialized = false
-  private var isMaster_ = false
+  private var _isDriver = false
 
   private var stopBroadcast = false
 
   private var trackMV: TrackMultipleValues = null
 
-  def initialize(isMaster__ : Boolean) {
+  def initialize(__isDriver: Boolean) {
     synchronized {
       if (!initialized) {
+        _isDriver = __isDriver
 
-        isMaster_ = isMaster__
-
-        if (isMaster) {
+        if (isDriver) {
           trackMV = new TrackMultipleValues
           trackMV.setDaemon(true)
           trackMV.start()
         
-          // Set masterHostAddress to the master's IP address for the slaves to read
-          System.setProperty("spark.MultiTracker.MasterHostAddress", Utils.localIpAddress)
+          // Set DriverHostAddress to the driver's IP address for the slaves to read
+          System.setProperty("spark.MultiTracker.DriverHostAddress", Utils.localIpAddress)
         }
 
         initialized = true
@@ -54,10 +53,10 @@ extends Logging {
   }
 
   // Load common parameters
-  private var MasterHostAddress_ = System.getProperty(
-    "spark.MultiTracker.MasterHostAddress", "")
-  private var MasterTrackerPort_ = System.getProperty(
-    "spark.broadcast.masterTrackerPort", "11111").toInt
+  private var DriverHostAddress_ = System.getProperty(
+    "spark.MultiTracker.DriverHostAddress", "")
+  private var DriverTrackerPort_ = System.getProperty(
+    "spark.broadcast.driverTrackerPort", "11111").toInt
   private var BlockSize_ = System.getProperty(
     "spark.broadcast.blockSize", "4096").toInt * 1024
   private var MaxRetryCount_ = System.getProperty(
@@ -91,11 +90,11 @@ extends Logging {
   private var EndGameFraction_ = System.getProperty(
       "spark.broadcast.endGameFraction", "0.95").toDouble
 
-  def isMaster = isMaster_
+  def isDriver = _isDriver
 
   // Common config params
-  def MasterHostAddress = MasterHostAddress_
-  def MasterTrackerPort = MasterTrackerPort_
+  def DriverHostAddress = DriverHostAddress_
+  def DriverTrackerPort = DriverTrackerPort_
   def BlockSize = BlockSize_
   def MaxRetryCount = MaxRetryCount_
 
@@ -123,7 +122,7 @@ extends Logging {
       var threadPool = Utils.newDaemonCachedThreadPool()
       var serverSocket: ServerSocket = null
 
-      serverSocket = new ServerSocket(MasterTrackerPort)
+      serverSocket = new ServerSocket(DriverTrackerPort)
       logInfo("TrackMultipleValues started at " + serverSocket)
 
       try {
@@ -235,7 +234,7 @@ extends Logging {
       try {
         // Connect to the tracker to find out GuideInfo
         clientSocketToTracker =
-          new Socket(MultiTracker.MasterHostAddress, MultiTracker.MasterTrackerPort)
+          new Socket(MultiTracker.DriverHostAddress, MultiTracker.DriverTrackerPort)
         oosTracker =
           new ObjectOutputStream(clientSocketToTracker.getOutputStream)
         oosTracker.flush()
@@ -276,7 +275,7 @@ extends Logging {
   }
   
   def registerBroadcast(id: Long, gInfo: SourceInfo) {
-    val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort)
+    val socket = new Socket(MultiTracker.DriverHostAddress, DriverTrackerPort)
     val oosST = new ObjectOutputStream(socket.getOutputStream)
     oosST.flush()
     val oisST = new ObjectInputStream(socket.getInputStream)
@@ -303,7 +302,7 @@ extends Logging {
   }
 
   def unregisterBroadcast(id: Long) {
-    val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort)
+    val socket = new Socket(MultiTracker.DriverHostAddress, DriverTrackerPort)
     val oosST = new ObjectOutputStream(socket.getOutputStream)
     oosST.flush()
     val oisST = new ObjectInputStream(socket.getInputStream)
diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
index f573512835d03ce5f807847fa8d3735a2927046d..c55c4761172dbdd4c5b2f5e607cf8d4d888f81d6 100644
--- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
@@ -98,7 +98,7 @@ extends Broadcast[T](id) with Logging with Serializable {
 
         case None =>
           logInfo("Started reading broadcast variable " + id)
-          // Initializing everything because Master will only send null/0 values
+          // Initializing everything because Driver will only send null/0 values
           // Only the 1st worker in a node can be here. Others will get from cache
           initializeWorkerVariables()
 
@@ -157,55 +157,55 @@ extends Broadcast[T](id) with Logging with Serializable {
       listenPortLock.synchronized { listenPortLock.wait() }
     }
 
-    var clientSocketToMaster: Socket = null
-    var oosMaster: ObjectOutputStream = null
-    var oisMaster: ObjectInputStream = null
+    var clientSocketToDriver: Socket = null
+    var oosDriver: ObjectOutputStream = null
+    var oisDriver: ObjectInputStream = null
 
     // Connect and receive broadcast from the specified source, retrying the
     // specified number of times in case of failures
     var retriesLeft = MultiTracker.MaxRetryCount
     do {
-      // Connect to Master and send this worker's Information
-      clientSocketToMaster = new Socket(MultiTracker.MasterHostAddress, gInfo.listenPort)
-      oosMaster = new ObjectOutputStream(clientSocketToMaster.getOutputStream)
-      oosMaster.flush()
-      oisMaster = new ObjectInputStream(clientSocketToMaster.getInputStream)
+      // Connect to Driver and send this worker's Information
+      clientSocketToDriver = new Socket(MultiTracker.DriverHostAddress, gInfo.listenPort)
+      oosDriver = new ObjectOutputStream(clientSocketToDriver.getOutputStream)
+      oosDriver.flush()
+      oisDriver = new ObjectInputStream(clientSocketToDriver.getInputStream)
 
-      logDebug("Connected to Master's guiding object")
+      logDebug("Connected to Driver's guiding object")
 
       // Send local source information
-      oosMaster.writeObject(SourceInfo(hostAddress, listenPort))
-      oosMaster.flush()
+      oosDriver.writeObject(SourceInfo(hostAddress, listenPort))
+      oosDriver.flush()
 
-      // Receive source information from Master
-      var sourceInfo = oisMaster.readObject.asInstanceOf[SourceInfo]
+      // Receive source information from Driver
+      var sourceInfo = oisDriver.readObject.asInstanceOf[SourceInfo]
       totalBlocks = sourceInfo.totalBlocks
       arrayOfBlocks = new Array[BroadcastBlock](totalBlocks)
       totalBlocksLock.synchronized { totalBlocksLock.notifyAll() }
       totalBytes = sourceInfo.totalBytes
 
-      logDebug("Received SourceInfo from Master:" + sourceInfo + " My Port: " + listenPort)
+      logDebug("Received SourceInfo from Driver:" + sourceInfo + " My Port: " + listenPort)
 
       val start = System.nanoTime
       val receptionSucceeded = receiveSingleTransmission(sourceInfo)
       val time = (System.nanoTime - start) / 1e9
 
-      // Updating some statistics in sourceInfo. Master will be using them later
+      // Updating some statistics in sourceInfo. Driver will be using them later
       if (!receptionSucceeded) {
         sourceInfo.receptionFailed = true
       }
 
-      // Send back statistics to the Master
-      oosMaster.writeObject(sourceInfo)
+      // Send back statistics to the Driver
+      oosDriver.writeObject(sourceInfo)
 
-      if (oisMaster != null) {
-        oisMaster.close()
+      if (oisDriver != null) {
+        oisDriver.close()
       }
-      if (oosMaster != null) {
-        oosMaster.close()
+      if (oosDriver != null) {
+        oosDriver.close()
       }
-      if (clientSocketToMaster != null) {
-        clientSocketToMaster.close()
+      if (clientSocketToDriver != null) {
+        clientSocketToDriver.close()
       }
 
       retriesLeft -= 1
@@ -552,7 +552,7 @@ extends Broadcast[T](id) with Logging with Serializable {
       }
 
       private def sendObject() {
-        // Wait till receiving the SourceInfo from Master
+        // Wait till receiving the SourceInfo from Driver
         while (totalBlocks == -1) {
           totalBlocksLock.synchronized { totalBlocksLock.wait() }
         }
@@ -576,7 +576,7 @@ extends Broadcast[T](id) with Logging with Serializable {
 
 private[spark] class TreeBroadcastFactory
 extends BroadcastFactory {
-  def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) }
+  def initialize(isDriver: Boolean) { MultiTracker.initialize(isDriver) }
 
   def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
     new TreeBroadcast[T](value_, isLocal, id)
diff --git a/core/src/main/scala/spark/deploy/JobDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
similarity index 51%
rename from core/src/main/scala/spark/deploy/JobDescription.scala
rename to core/src/main/scala/spark/deploy/ApplicationDescription.scala
index 20879c5f1110a6806aa064d9b2f64ef6b32f7c70..6659e53b25f370f1b6747a03b7a42ec147b121de 100644
--- a/core/src/main/scala/spark/deploy/JobDescription.scala
+++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
@@ -1,13 +1,14 @@
 package spark.deploy
 
-private[spark] class JobDescription(
+private[spark] class ApplicationDescription(
     val name: String,
     val cores: Int,
     val memoryPerSlave: Int,
-    val command: Command)
+    val command: Command,
+    val sparkHome: String)
   extends Serializable {
 
   val user = System.getProperty("user.name", "<unknown>")
 
-  override def toString: String = "JobDescription(" + name + ")"
+  override def toString: String = "ApplicationDescription(" + name + ")"
 }
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala
index 457122745b61847739080681da0aa429dadc0a10..8a3e64e4c22fa60ef4bf05a02a14ff399aea2632 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/spark/deploy/DeployMessage.scala
@@ -1,10 +1,9 @@
 package spark.deploy
 
 import spark.deploy.ExecutorState.ExecutorState
-import spark.deploy.master.{WorkerInfo, JobInfo}
+import spark.deploy.master.{WorkerInfo, ApplicationInfo}
 import spark.deploy.worker.ExecutorRunner
 import scala.collection.immutable.List
-import scala.collection.mutable.HashMap
 
 
 private[spark] sealed trait DeployMessage extends Serializable
@@ -24,36 +23,39 @@ case class RegisterWorker(
 
 private[spark] 
 case class ExecutorStateChanged(
-    jobId: String,
+    appId: String,
     execId: Int,
     state: ExecutorState,
     message: Option[String],
     exitStatus: Option[Int])
   extends DeployMessage
 
+private[spark] case class Heartbeat(workerId: String) extends DeployMessage
+
 // Master to Worker
 
 private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
 private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage
-private[spark] case class KillExecutor(jobId: String, execId: Int) extends DeployMessage
+private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage
 
 private[spark] case class LaunchExecutor(
-    jobId: String,
+    appId: String,
     execId: Int,
-    jobDesc: JobDescription,
+    appDesc: ApplicationDescription,
     cores: Int,
-    memory: Int)
+    memory: Int,
+    sparkHome: String)
   extends DeployMessage
 
-
 // Client to Master
 
-private[spark] case class RegisterJob(jobDescription: JobDescription) extends DeployMessage
+private[spark] case class RegisterApplication(appDescription: ApplicationDescription)
+  extends DeployMessage
 
 // Master to Client
 
 private[spark] 
-case class RegisteredJob(jobId: String) extends DeployMessage
+case class RegisteredApplication(appId: String) extends DeployMessage
 
 private[spark] 
 case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int)
@@ -63,7 +65,7 @@ case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String
                            exitStatus: Option[Int])
 
 private[spark]
-case class JobKilled(message: String)
+case class ApplicationRemoved(message: String)
 
 // Internal message in Client
 
@@ -76,8 +78,11 @@ private[spark] case object RequestMasterState
 // Master to MasterWebUI
 
 private[spark] 
-case class MasterState(uri: String, workers: Array[WorkerInfo], activeJobs: Array[JobInfo],
-  completedJobs: Array[JobInfo])
+case class MasterState(host: String, port: Int, workers: Array[WorkerInfo],
+  activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
+
+  def uri = "spark://" + host + ":" + port
+}
 
 //  WorkerWebUI to Worker
 private[spark] case object RequestWorkerState
@@ -85,6 +90,6 @@ private[spark] case object RequestWorkerState
 // Worker to WorkerWebUI
 
 private[spark]
-case class WorkerState(uri: String, workerId: String, executors: List[ExecutorRunner], 
+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)
diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala
index 12069947fc5ad3c38ee73e170993894d247a096c..702defb628d7ac341167db20311b0d58374e7067 100644
--- a/core/src/main/scala/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala
@@ -1,6 +1,6 @@
 package spark.deploy
 
-import master.{JobInfo, WorkerInfo}
+import master.{ApplicationInfo, WorkerInfo}
 import worker.ExecutorRunner
 import spray.json._
 
@@ -20,8 +20,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
     )
   }
 
-  implicit object JobInfoJsonFormat extends RootJsonWriter[JobInfo] {
-    def write(obj: JobInfo) = JsObject(
+  implicit object AppInfoJsonFormat extends RootJsonWriter[ApplicationInfo] {
+    def write(obj: ApplicationInfo) = JsObject(
       "starttime" -> JsNumber(obj.startTime),
       "id" -> JsString(obj.id),
       "name" -> JsString(obj.desc.name),
@@ -31,8 +31,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
       "submitdate" -> JsString(obj.submitDate.toString))
   }
 
-  implicit object JobDescriptionJsonFormat extends RootJsonWriter[JobDescription] {
-    def write(obj: JobDescription) = JsObject(
+  implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] {
+    def write(obj: ApplicationDescription) = JsObject(
       "name" -> JsString(obj.name),
       "cores" -> JsNumber(obj.cores),
       "memoryperslave" -> JsNumber(obj.memoryPerSlave),
@@ -44,8 +44,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
     def write(obj: ExecutorRunner) = JsObject(
       "id" -> JsNumber(obj.execId),
       "memory" -> JsNumber(obj.memory),
-      "jobid" -> JsString(obj.jobId),
-      "jobdesc" -> obj.jobDesc.toJson.asJsObject
+      "appid" -> JsString(obj.appId),
+      "appdesc" -> obj.appDesc.toJson.asJsObject
     )
   }
 
@@ -57,8 +57,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
       "coresused" -> JsNumber(obj.workers.map(_.coresUsed).sum),
       "memory" -> JsNumber(obj.workers.map(_.memory).sum),
       "memoryused" -> JsNumber(obj.workers.map(_.memoryUsed).sum),
-      "activejobs" -> JsArray(obj.activeJobs.toList.map(_.toJson)),
-      "completedjobs" -> JsArray(obj.completedJobs.toList.map(_.toJson))
+      "activeapps" -> JsArray(obj.activeApps.toList.map(_.toJson)),
+      "completedapps" -> JsArray(obj.completedApps.toList.map(_.toJson))
     )
   }
 
diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
index 4211d805967a9087e19a7f0285d9efdad776913c..22319a96caef7ff80f97259bccd8381b0f6514bd 100644
--- a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
@@ -9,43 +9,32 @@ import spark.{Logging, Utils}
 
 import scala.collection.mutable.ArrayBuffer
 
+/**
+ * Testing class that creates a Spark standalone process in-cluster (that is, running the
+ * spark.deploy.master.Master and spark.deploy.worker.Workers in the same JVMs). Executors launched
+ * by the Workers still run in separate JVMs. This can be used to test distributed operation and
+ * fault recovery without spinning up a lot of processes.
+ */
 private[spark]
-class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) extends Logging {
+class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging {
   
-  val localIpAddress = Utils.localIpAddress
+  private val localIpAddress = Utils.localIpAddress
+  private val masterActorSystems = ArrayBuffer[ActorSystem]()
+  private val workerActorSystems = ArrayBuffer[ActorSystem]()
   
-  var masterActor : ActorRef = _
-  var masterActorSystem : ActorSystem = _
-  var masterPort : Int = _
-  var masterUrl : String = _
-  
-  val slaveActorSystems = ArrayBuffer[ActorSystem]()
-  val slaveActors = ArrayBuffer[ActorRef]()
-  
-  def start() : String = {
-    logInfo("Starting a local Spark cluster with " + numSlaves + " slaves.")
+  def start(): String = {
+    logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
 
     /* Start the Master */
-    val (actorSystem, masterPort) = AkkaUtils.createActorSystem("sparkMaster", localIpAddress, 0)
-    masterActorSystem = actorSystem
-    masterUrl = "spark://" + localIpAddress + ":" + masterPort
-    val actor = masterActorSystem.actorOf(
-      Props(new Master(localIpAddress, masterPort, 0)), name = "Master")
-    masterActor = actor
-
-    /* Start the Slaves */
-    for (slaveNum <- 1 to numSlaves) {
-      /* We can pretend to test distributed stuff by giving the slaves distinct hostnames.
-         All of 127/8 should be a loopback, we use 127.100.*.* in hopes that it is
-         sufficiently distinctive. */
-      val slaveIpAddress = "127.100.0." + (slaveNum % 256)
-      val (actorSystem, boundPort) = 
-        AkkaUtils.createActorSystem("sparkWorker" + slaveNum, slaveIpAddress, 0)
-      slaveActorSystems += actorSystem
-      val actor = actorSystem.actorOf(
-        Props(new Worker(slaveIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)),
-        name = "Worker")
-      slaveActors += actor
+    val (masterSystem, masterPort) = Master.startSystemAndActor(localIpAddress, 0, 0)
+    masterActorSystems += masterSystem
+    val masterUrl = "spark://" + localIpAddress + ":" + masterPort
+
+    /* Start the Workers */
+    for (workerNum <- 1 to numWorkers) {
+      val (workerSystem, _) = Worker.startSystemAndActor(localIpAddress, 0, 0, coresPerWorker,
+        memoryPerWorker, masterUrl, null, Some(workerNum))
+      workerActorSystems += workerSystem
     }
 
     return masterUrl
@@ -53,10 +42,10 @@ class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int)
 
   def stop() {
     logInfo("Shutting down local Spark cluster.")
-    // Stop the slaves before the master so they don't get upset that it disconnected
-    slaveActorSystems.foreach(_.shutdown())
-    slaveActorSystems.foreach(_.awaitTermination())
-    masterActorSystem.shutdown()
-    masterActorSystem.awaitTermination()
+    // Stop the workers before the master so they don't get upset that it disconnected
+    workerActorSystems.foreach(_.shutdown())
+    workerActorSystems.foreach(_.awaitTermination())
+    masterActorSystems.foreach(_.shutdown())
+    masterActorSystems.foreach(_.awaitTermination())
   }
 }
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala
index 0aee7597962af39883ff16c3fbf3b6bd76e14555..a38218a3913d350a9768114d2d8026f7d5c84d56 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/spark/deploy/client/Client.scala
@@ -8,30 +8,25 @@ import akka.pattern.AskTimeoutException
 import spark.{SparkException, Logging}
 import akka.remote.RemoteClientLifeCycleEvent
 import akka.remote.RemoteClientShutdown
-import spark.deploy.RegisterJob
+import spark.deploy.RegisterApplication
+import spark.deploy.master.Master
 import akka.remote.RemoteClientDisconnected
 import akka.actor.Terminated
 import scala.concurrent.Await
 
 /**
- * The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description,
- * and a listener for job events, and calls back the listener when various events occur.
+ * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
+ * and a listener for cluster events, and calls back the listener when various events occur.
  */
 private[spark] class Client(
     actorSystem: ActorSystem,
     masterUrl: String,
-    jobDescription: JobDescription,
+    appDescription: ApplicationDescription,
     listener: ClientListener)
   extends Logging {
 
-  val MASTER_REGEX = "spark://([^:]+):([0-9]+)".r
-
   var actor: ActorRef = null
-  var jobId: String = null
-
-  if (MASTER_REGEX.unapplySeq(masterUrl) == None) {
-    throw new SparkException("Invalid master URL: " + masterUrl)
-  }
+  var appId: String = null
 
   class ClientActor extends Actor with Logging {
     var master: ActorRef = null
@@ -39,13 +34,11 @@ private[spark] class Client(
     var alreadyDisconnected = false  // To avoid calling listener.disconnected() multiple times
 
     override def preStart() {
-      val Seq(masterHost, masterPort) = MASTER_REGEX.unapplySeq(masterUrl).get
-      logInfo("Connecting to master spark://" + masterHost + ":" + masterPort)
-      val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort)
+      logInfo("Connecting to master " + masterUrl)
       try {
-        master = context.actorFor(akkaUrl)
+        master = context.actorFor(Master.toAkkaUrl(masterUrl))
         masterAddress = master.path.address
-        master ! RegisterJob(jobDescription)
+        master ! RegisterApplication(appDescription)
         context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
         context.watch(master)  // Doesn't work with remote actors, but useful for testing
       } catch {
@@ -57,17 +50,22 @@ private[spark] class Client(
     }
 
     override def receive = {
-      case RegisteredJob(jobId_) =>
-        jobId = jobId_
-        listener.connected(jobId)
+      case RegisteredApplication(appId_) =>
+        appId = appId_
+        listener.connected(appId)
+
+      case ApplicationRemoved(message) =>
+        logError("Master removed our application: %s; stopping client".format(message))
+        markDisconnected()
+        context.stop(self)
 
       case ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) =>
-        val fullId = jobId + "/" + id
+        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)
 
       case ExecutorUpdated(id, state, message, exitStatus) =>
-        val fullId = jobId + "/" + id
+        val fullId = appId + "/" + id
         val messageText = message.map(s => " (" + s + ")").getOrElse("")
         logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText))
         if (ExecutorState.isFinished(state)) {
@@ -114,7 +112,7 @@ private[spark] class Client(
   def stop() {
     if (actor != null) {
       try {
-        val timeout = 1.seconds
+        val timeout = 5.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 da6abcc9c26c083482eaaac5f002e2151d4803b1..b7008321df564976d37ca9f428d7d47920a30bf3 100644
--- a/core/src/main/scala/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala
@@ -8,11 +8,11 @@ package spark.deploy.client
  * Users of this API should *not* block inside the callback methods.
  */
 private[spark] trait ClientListener {
-  def connected(jobId: String): Unit
+  def connected(appId: String): Unit
 
   def disconnected(): Unit
 
-  def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int): Unit
+  def executorAdded(fullId: String, workerId: String, host: String, cores: Int, memory: Int): Unit
 
-  def executorRemoved(id: String, message: String, exitStatus: Option[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 57a7e123b78a5449802469eb9bab7bb21753020d..dc004b59ca5ac247d4e7c8125b775a1f1698e7ea 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/spark/deploy/client/TestClient.scala
@@ -2,13 +2,13 @@ package spark.deploy.client
 
 import spark.util.AkkaUtils
 import spark.{Logging, Utils}
-import spark.deploy.{Command, JobDescription}
+import spark.deploy.{Command, ApplicationDescription}
 
 private[spark] object TestClient {
 
   class TestListener extends ClientListener with Logging {
     def connected(id: String) {
-      logInfo("Connected to master, got job ID " + id)
+      logInfo("Connected to master, got app ID " + id)
     }
 
     def disconnected() {
@@ -24,8 +24,8 @@ private[spark] object TestClient {
   def main(args: Array[String]) {
     val url = args(0)
     val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
-    val desc = new JobDescription(
-      "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()))
+    val desc = new ApplicationDescription(
+      "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home")
     val listener = new TestListener
     val client = new Client(actorSystem, url, desc, listener)
     client.start()
diff --git a/core/src/main/scala/spark/deploy/master/JobInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
similarity index 82%
rename from core/src/main/scala/spark/deploy/master/JobInfo.scala
rename to core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
index 130b031a2af6cea087b6c2ffb995103ffa7beb40..3591a9407237a765003af9d59d99603c70cf06a8 100644
--- a/core/src/main/scala/spark/deploy/master/JobInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
@@ -1,18 +1,18 @@
 package spark.deploy.master
 
-import spark.deploy.JobDescription
+import spark.deploy.ApplicationDescription
 import java.util.Date
 import akka.actor.ActorRef
 import scala.collection.mutable
 
-private[spark] class JobInfo(
+private[spark] class ApplicationInfo(
     val startTime: Long,
     val id: String,
-    val desc: JobDescription,
+    val desc: ApplicationDescription,
     val submitDate: Date,
-    val actor: ActorRef)
+    val driver: ActorRef)
 {
-  var state = JobState.WAITING
+  var state = ApplicationState.WAITING
   var executors = new mutable.HashMap[Int, ExecutorInfo]
   var coresGranted = 0
   var endTime = -1L
@@ -48,7 +48,7 @@ private[spark] class JobInfo(
     _retryCount
   }
 
-  def markFinished(endState: JobState.Value) {
+  def markFinished(endState: ApplicationState.Value) {
     state = endState
     endTime = System.currentTimeMillis()
   }
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala
new file mode 100644
index 0000000000000000000000000000000000000000..15016b388d2d82aaf12ce936122ede712c38bade
--- /dev/null
+++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala
@@ -0,0 +1,11 @@
+package spark.deploy.master
+
+private[spark] object ApplicationState
+  extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
+
+  type ApplicationState = Value
+
+  val WAITING, RUNNING, FINISHED, FAILED = Value
+
+  val MAX_NUM_RETRY = 10
+}
diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
index 1db2c326333f1751f22ff83d215b277020747718..48e6055fb572aefd1b71b5bf40fd838d86fe4b0a 100644
--- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
@@ -4,12 +4,12 @@ import spark.deploy.ExecutorState
 
 private[spark] class ExecutorInfo(
     val id: Int,
-    val job: JobInfo,
+    val application: ApplicationInfo,
     val worker: WorkerInfo,
     val cores: Int,
     val memory: Int) {
 
   var state = ExecutorState.LAUNCHING
 
-  def fullId: String = job.id + "/" + id
+  def fullId: String = application.id + "/" + id
 }
diff --git a/core/src/main/scala/spark/deploy/master/JobState.scala b/core/src/main/scala/spark/deploy/master/JobState.scala
deleted file mode 100644
index 2b70cf01918d7b971fccd4c6859536ff8c323e40..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/deploy/master/JobState.scala
+++ /dev/null
@@ -1,9 +0,0 @@
-package spark.deploy.master
-
-private[spark] object JobState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
-  type JobState = Value
-
-  val WAITING, RUNNING, FINISHED, FAILED = Value
-
-  val MAX_NUM_RETRY = 10
-}
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index e034312c12ba19ed752e47170da2f5a12bd377d5..e070a15a54f06abcb892d3a3844864a1061245b3 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -3,6 +3,7 @@ package spark.deploy.master
 import akka.actor._
 import akka.actor.Terminated
 import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+import akka.util.duration._
 
 import java.text.SimpleDateFormat
 import java.util.Date
@@ -15,21 +16,24 @@ import spark.util.AkkaUtils
 
 
 private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
-  val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For job IDs
+  val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For application IDs
+  val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
 
-  var nextJobNumber = 0
+  var nextAppNumber = 0
   val workers = new HashSet[WorkerInfo]
   val idToWorker = new HashMap[String, WorkerInfo]
   val actorToWorker = new HashMap[ActorRef, WorkerInfo]
   val addressToWorker = new HashMap[Address, WorkerInfo]
 
-  val jobs = new HashSet[JobInfo]
-  val idToJob = new HashMap[String, JobInfo]
-  val actorToJob = new HashMap[ActorRef, JobInfo]
-  val addressToJob = new HashMap[Address, JobInfo]
+  val apps = new HashSet[ApplicationInfo]
+  val idToApp = new HashMap[String, ApplicationInfo]
+  val actorToApp = new HashMap[ActorRef, ApplicationInfo]
+  val addressToApp = new HashMap[Address, ApplicationInfo]
 
-  val waitingJobs = new ArrayBuffer[JobInfo]
-  val completedJobs = new ArrayBuffer[JobInfo]
+  val waitingApps = new ArrayBuffer[ApplicationInfo]
+  val completedApps = new ArrayBuffer[ApplicationInfo]
+
+  var firstApp: Option[ApplicationInfo] = None
 
   val masterPublicAddress = {
     val envVar = System.getenv("SPARK_PUBLIC_DNS")
@@ -37,15 +41,16 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
   }
 
   // As a temporary workaround before better ways of configuring memory, we allow users to set
-  // a flag that will perform round-robin scheduling across the nodes (spreading out each job
-  // among all the nodes) instead of trying to consolidate each job onto a small # of nodes.
-  val spreadOutJobs = System.getProperty("spark.deploy.spreadOut", "false").toBoolean
+  // a flag that will perform round-robin scheduling across the nodes (spreading out each app
+  // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
+  val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
     logInfo("Starting Spark master at spark://" + ip + ":" + port)
     // Listen for remote client disconnection events, since they don't go through Akka's watch()
     context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
     startWebUi()
+    context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis)(timeOutDeadWorkers())
   }
 
   def startWebUi() {
@@ -73,94 +78,101 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
       }
     }
 
-    case RegisterJob(description) => {
-      logInfo("Registering job " + description.name)
-      val job = addJob(description, sender)
-      logInfo("Registered job " + description.name + " with ID " + job.id)
-      waitingJobs += job
+    case RegisterApplication(description) => {
+      logInfo("Registering app " + description.name)
+      val app = addApplication(description, sender)
+      logInfo("Registered app " + description.name + " with ID " + app.id)
+      waitingApps += app
       context.watch(sender)  // This doesn't work with remote actors but helps for testing
-      sender ! RegisteredJob(job.id)
+      sender ! RegisteredApplication(app.id)
       schedule()
     }
 
-    case ExecutorStateChanged(jobId, execId, state, message, exitStatus) => {
-      val execOption = idToJob.get(jobId).flatMap(job => job.executors.get(execId))
+    case ExecutorStateChanged(appId, execId, state, message, exitStatus) => {
+      val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId))
       execOption match {
         case Some(exec) => {
           exec.state = state
-          exec.job.actor ! ExecutorUpdated(execId, state, message, exitStatus)
+          exec.application.driver ! ExecutorUpdated(execId, state, message, exitStatus)
           if (ExecutorState.isFinished(state)) {
-            val jobInfo = idToJob(jobId)
-            // Remove this executor from the worker and job
+            val appInfo = idToApp(appId)
+            // Remove this executor from the worker and app
             logInfo("Removing executor " + exec.fullId + " because it is " + state)
-            jobInfo.removeExecutor(exec)
+            appInfo.removeExecutor(exec)
             exec.worker.removeExecutor(exec)
 
             // Only retry certain number of times so we don't go into an infinite loop.
-            if (jobInfo.incrementRetryCount <= JobState.MAX_NUM_RETRY) {
+            if (appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) {
               schedule()
             } else {
-              val e = new SparkException("Job %s wth ID %s failed %d times.".format(
-                jobInfo.desc.name, jobInfo.id, jobInfo.retryCount))
-              logError(e.getMessage, e)
-              throw e
-              //System.exit(1)
+              logError("Application %s with ID %s failed %d times, removing it".format(
+                appInfo.desc.name, appInfo.id, appInfo.retryCount))
+              removeApplication(appInfo, ApplicationState.FAILED)
             }
           }
         }
         case None =>
-          logWarning("Got status update for unknown executor " + jobId + "/" + execId)
+          logWarning("Got status update for unknown executor " + appId + "/" + execId)
+      }
+    }
+
+    case Heartbeat(workerId) => {
+      idToWorker.get(workerId) match {
+        case Some(workerInfo) =>
+          workerInfo.lastHeartbeat = System.currentTimeMillis()
+        case None =>
+          logWarning("Got heartbeat from unregistered worker " + workerId)
       }
     }
 
     case Terminated(actor) => {
-      // The disconnected actor could've been either a worker or a job; remove whichever of
+      // The disconnected actor could've been either a worker or an app; remove whichever of
       // those we have an entry for in the corresponding actor hashmap
       actorToWorker.get(actor).foreach(removeWorker)
-      actorToJob.get(actor).foreach(removeJob)
+      actorToApp.get(actor).foreach(finishApplication)
     }
 
     case RemoteClientDisconnected(transport, address) => {
-      // The disconnected client could've been either a worker or a job; remove whichever it was
+      // The disconnected client could've been either a worker or an app; remove whichever it was
       addressToWorker.get(address).foreach(removeWorker)
-      addressToJob.get(address).foreach(removeJob)
+      addressToApp.get(address).foreach(finishApplication)
     }
 
     case RemoteClientShutdown(transport, address) => {
-      // The disconnected client could've been either a worker or a job; remove whichever it was
+      // The disconnected client could've been either a worker or an app; remove whichever it was
       addressToWorker.get(address).foreach(removeWorker)
-      addressToJob.get(address).foreach(removeJob)
+      addressToApp.get(address).foreach(finishApplication)
     }
 
     case RequestMasterState => {
-      sender ! MasterState(ip + ":" + port, workers.toArray, jobs.toArray, completedJobs.toArray)
+      sender ! MasterState(ip, port, workers.toArray, apps.toArray, completedApps.toArray)
     }
   }
 
   /**
-   * Can a job use the given worker? True if the worker has enough memory and we haven't already
-   * launched an executor for the job on it (right now the standalone backend doesn't like having
+   * Can an app use the given worker? True if the worker has enough memory and we haven't already
+   * launched an executor for the app on it (right now the standalone backend doesn't like having
    * two executors on the same worker).
    */
-  def canUse(job: JobInfo, worker: WorkerInfo): Boolean = {
-    worker.memoryFree >= job.desc.memoryPerSlave && !worker.hasExecutor(job)
+  def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = {
+    worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app)
   }
 
   /**
-   * Schedule the currently available resources among waiting jobs. This method will be called
-   * every time a new job joins or resource availability changes.
+   * Schedule the currently available resources among waiting apps. This method will be called
+   * every time a new app joins or resource availability changes.
    */
   def schedule() {
-    // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first job
-    // in the queue, then the second job, etc.
-    if (spreadOutJobs) {
-      // Try to spread out each job among all the nodes, until it has all its cores
-      for (job <- waitingJobs if job.coresLeft > 0) {
+    // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
+    // in the queue, then the second app, etc.
+    if (spreadOutApps) {
+      // Try to spread out each app among all the nodes, until it has all its cores
+      for (app <- waitingApps if app.coresLeft > 0) {
         val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE)
-          .filter(canUse(job, _)).sortBy(_.coresFree).reverse
+                                   .filter(canUse(app, _)).sortBy(_.coresFree).reverse
         val numUsable = usableWorkers.length
         val assigned = new Array[Int](numUsable) // Number of cores to give on each node
-        var toAssign = math.min(job.coresLeft, usableWorkers.map(_.coresFree).sum)
+        var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum)
         var pos = 0
         while (toAssign > 0) {
           if (usableWorkers(pos).coresFree - assigned(pos) > 0) {
@@ -172,22 +184,22 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
         // Now that we've decided how many cores to give on each node, let's actually give them
         for (pos <- 0 until numUsable) {
           if (assigned(pos) > 0) {
-            val exec = job.addExecutor(usableWorkers(pos), assigned(pos))
-            launchExecutor(usableWorkers(pos), exec)
-            job.state = JobState.RUNNING
+            val exec = app.addExecutor(usableWorkers(pos), assigned(pos))
+            launchExecutor(usableWorkers(pos), exec, app.desc.sparkHome)
+            app.state = ApplicationState.RUNNING
           }
         }
       }
     } else {
-      // Pack each job into as few nodes as possible until we've assigned all its cores
-      for (worker <- workers if worker.coresFree > 0) {
-        for (job <- waitingJobs if job.coresLeft > 0) {
-          if (canUse(job, worker)) {
-            val coresToUse = math.min(worker.coresFree, job.coresLeft)
+      // Pack each app into as few nodes as possible until we've assigned all its cores
+      for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) {
+        for (app <- waitingApps if app.coresLeft > 0) {
+          if (canUse(app, worker)) {
+            val coresToUse = math.min(worker.coresFree, app.coresLeft)
             if (coresToUse > 0) {
-              val exec = job.addExecutor(worker, coresToUse)
-              launchExecutor(worker, exec)
-              job.state = JobState.RUNNING
+              val exec = app.addExecutor(worker, coresToUse)
+              launchExecutor(worker, exec, app.desc.sparkHome)
+              app.state = ApplicationState.RUNNING
             }
           }
         }
@@ -195,11 +207,11 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
     }
   }
 
-  def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo) {
+  def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
     logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
     worker.addExecutor(exec)
-    worker.actor ! LaunchExecutor(exec.job.id, exec.id, exec.job.desc, exec.cores, exec.memory)
-    exec.job.actor ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory)
+    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)
   }
 
   def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int,
@@ -221,54 +233,97 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
     actorToWorker -= worker.actor
     addressToWorker -= worker.actor.path.address
     for (exec <- worker.executors.values) {
-      exec.job.actor ! ExecutorStateChanged(exec.job.id, exec.id, ExecutorState.LOST, None, None)
-      exec.job.executors -= exec.id
+      logInfo("Telling app of lost executor: " + exec.id)
+      exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None)
+      exec.application.removeExecutor(exec)
     }
   }
 
-  def addJob(desc: JobDescription, actor: ActorRef): JobInfo = {
+  def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
     val now = System.currentTimeMillis()
     val date = new Date(now)
-    val job = new JobInfo(now, newJobId(date), desc, date, actor)
-    jobs += job
-    idToJob(job.id) = job
-    actorToJob(sender) = job
-    addressToJob(sender.path.address) = job
-    return job
+    val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver)
+    apps += app
+    idToApp(app.id) = app
+    actorToApp(driver) = app
+    addressToApp(driver.path.address) = app
+    if (firstApp == None) {
+      firstApp = Some(app)
+    }
+    val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray
+    if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= desc.memoryPerSlave)) {
+      logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
+    }
+    return app
+  }
+
+  def finishApplication(app: ApplicationInfo) {
+    removeApplication(app, ApplicationState.FINISHED)
   }
 
-  def removeJob(job: JobInfo) {
-    if (jobs.contains(job)) {
-      logInfo("Removing job " + job.id)
-      jobs -= job
-      idToJob -= job.id
-      actorToJob -= job.actor
-      addressToWorker -= job.actor.path.address
-      completedJobs += job   // Remember it in our history
-      waitingJobs -= job
-      for (exec <- job.executors.values) {
+  def removeApplication(app: ApplicationInfo, state: ApplicationState.Value) {
+    if (apps.contains(app)) {
+      logInfo("Removing app " + app.id)
+      apps -= app
+      idToApp -= app.id
+      actorToApp -= app.driver
+      addressToApp -= app.driver.path.address
+      completedApps += app   // Remember it in our history
+      waitingApps -= app
+      for (exec <- app.executors.values) {
         exec.worker.removeExecutor(exec)
-        exec.worker.actor ! KillExecutor(exec.job.id, exec.id)
+        exec.worker.actor ! KillExecutor(exec.application.id, exec.id)
       }
-      job.markFinished(JobState.FINISHED)  // TODO: Mark it as FAILED if it failed
+      app.markFinished(state)
+      app.driver ! ApplicationRemoved(state.toString)
       schedule()
     }
   }
 
-  /** Generate a new job ID given a job's submission date */
-  def newJobId(submitDate: Date): String = {
-    val jobId = "job-%s-%04d".format(DATE_FORMAT.format(submitDate), nextJobNumber)
-    nextJobNumber += 1
-    jobId
+  /** Generate a new app ID given a app's submission date */
+  def newApplicationId(submitDate: Date): String = {
+    val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber)
+    nextAppNumber += 1
+    appId
+  }
+
+  /** Check for, and remove, any timed-out workers */
+  def timeOutDeadWorkers() {
+    // Copy the workers into an array so we don't modify the hashset while iterating through it
+    val expirationTime = System.currentTimeMillis() - WORKER_TIMEOUT
+    val toRemove = workers.filter(_.lastHeartbeat < expirationTime).toArray
+    for (worker <- toRemove) {
+      logWarning("Removing %s because we got no heartbeat in %d seconds".format(
+        worker.id, WORKER_TIMEOUT))
+      removeWorker(worker)
+    }
   }
 }
 
 private[spark] object Master {
+  private val systemName = "sparkMaster"
+  private val actorName = "Master"
+  private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
+
   def main(argStrings: Array[String]) {
     val args = new MasterArguments(argStrings)
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port)
-    val actor = actorSystem.actorOf(
-      Props(new Master(args.ip, boundPort, args.webUiPort)), name = "Master")
+    val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort)
     actorSystem.awaitTermination()
   }
+
+  /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
+  def toAkkaUrl(sparkUrl: String): String = {
+    sparkUrl match {
+      case sparkUrlRegex(host, port) =>
+        "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
+      case _ =>
+        throw new SparkException("Invalid master URL: " + sparkUrl)
+    }
+  }
+
+  def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int) = {
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
+    val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName)
+    (actorSystem, boundPort)
+  }
 }
diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
index a4dadb6ef9484381121afb255da1d4f383c09945..59d59dde78032f7bc71ee8debc75cdf01d5097c9 100644
--- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
@@ -14,6 +14,9 @@ import spray.http.MediaTypes._
 import spark.deploy._
 import spark.deploy.JsonProtocol._
 
+/**
+ * Web UI server for the standalone master.
+ */
 private[spark]
 class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends Directives {
   import context.dispatcher
@@ -22,7 +25,7 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends
   val RESOURCE_DIR = "spark/deploy/master/webui"
   val STATIC_RESOURCE_DIR = "spark/deploy/static"
 
-  implicit val timeout = Timeout(1 seconds)
+  implicit val timeout = Timeout(10 seconds)
 
   val handler = {
     get {
@@ -40,19 +43,16 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends
             }
           }
       } ~
-      path("job") {
-        parameters("jobId", 'format ?) {
-          case (jobId, Some(js)) if (js.equalsIgnoreCase("json")) =>
+      path("app") {
+        parameters("appId", 'format ?) {
+          case (appId, Some(js)) if (js.equalsIgnoreCase("json")) =>
             val future = master ? RequestMasterState
-            val jobInfo = for (masterState <- future.mapTo[MasterState]) yield {
-              masterState.activeJobs.find(_.id == jobId) match {
-                case Some(job) => job
-                case _ => masterState.completedJobs.find(_.id == jobId) match {
-                  case Some(job) => job
-                  case _ => null
-                }
-              }
+            val appInfo = for (masterState <- future.mapTo[MasterState]) yield {
+              masterState.activeApps.find(_.id == appId).getOrElse({
+                masterState.completedApps.find(_.id == appId).getOrElse(null)
+              })
             }
+<<<<<<< HEAD
             respondWithMediaType(`application/json`) { ctx =>
               ctx.complete(jobInfo.mapTo[JobInfo])
             }
@@ -67,6 +67,20 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends
                     case _ => null
                   }
                 }
+=======
+            respondWithMediaType(MediaTypes.`application/json`) { ctx =>
+              ctx.complete(appInfo.mapTo[ApplicationInfo])
+            }
+          case (appId, _) =>
+            completeWith {
+              val future = master ? RequestMasterState
+              future.map { state =>
+                val masterState = state.asInstanceOf[MasterState]
+                val app = masterState.activeApps.find(_.id == appId).getOrElse({
+                  masterState.completedApps.find(_.id == appId).getOrElse(null)
+                })
+                spark.deploy.master.html.app_details.render(app)
+>>>>>>> 17e076de800ea0d4c55f2bd657348641f6f9c55b
               }
             }
         }
@@ -77,5 +91,4 @@ class MasterWebUI(master: ActorRef)(implicit val context: ActorContext) extends
       getFromResourceDirectory(RESOURCE_DIR)
     }
   }
-
 }
diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
index 5a7f5fef8a546812d20719af8d2a0f3dcab1af29..23df1bb463288721e38379023b3fd01c4c8632d8 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
@@ -18,6 +18,8 @@ private[spark] class WorkerInfo(
   var coresUsed = 0
   var memoryUsed = 0
 
+  var lastHeartbeat = System.currentTimeMillis()
+
   def coresFree: Int = cores - coresUsed
   def memoryFree: Int = memory - memoryUsed
 
@@ -35,8 +37,8 @@ private[spark] class WorkerInfo(
     }
   }
 
-  def hasExecutor(job: JobInfo): Boolean = {
-    executors.values.exists(_.job == job)
+  def hasExecutor(app: ApplicationInfo): Boolean = {
+    executors.values.exists(_.application == app)
   }
 
   def webUiAddress : String = {
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
index beceb55ecdf54750016af497b0689b5a37191a67..de11771c8e62d7cdb0b629a105bb8d1ca21e544c 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
@@ -1,7 +1,7 @@
 package spark.deploy.worker
 
 import java.io._
-import spark.deploy.{ExecutorState, ExecutorStateChanged, JobDescription}
+import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription}
 import akka.actor.ActorRef
 import spark.{Utils, Logging}
 import java.net.{URI, URL}
@@ -14,9 +14,9 @@ import spark.deploy.ExecutorStateChanged
  * Manages the execution of one executor process.
  */
 private[spark] class ExecutorRunner(
-    val jobId: String,
+    val appId: String,
     val execId: Int,
-    val jobDesc: JobDescription,
+    val appDesc: ApplicationDescription,
     val cores: Int,
     val memory: Int,
     val worker: ActorRef,
@@ -26,7 +26,7 @@ private[spark] class ExecutorRunner(
     val workDir: File)
   extends Logging {
 
-  val fullId = jobId + "/" + execId
+  val fullId = appId + "/" + execId
   var workerThread: Thread = null
   var process: Process = null
   var shutdownHook: Thread = null
@@ -60,24 +60,24 @@ private[spark] class ExecutorRunner(
         process.destroy()
         process.waitFor()
       }
-      worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None, None)
+      worker ! ExecutorStateChanged(appId, execId, ExecutorState.KILLED, None, None)
       Runtime.getRuntime.removeShutdownHook(shutdownHook)
     }
   }
 
-  /** Replace variables such as {{SLAVEID}} and {{CORES}} in a command argument passed to us */
+  /** Replace variables such as {{EXECUTOR_ID}} and {{CORES}} in a command argument passed to us */
   def substituteVariables(argument: String): String = argument match {
-    case "{{SLAVEID}}" => workerId
+    case "{{EXECUTOR_ID}}" => execId.toString
     case "{{HOSTNAME}}" => hostname
     case "{{CORES}}" => cores.toString
     case other => other
   }
 
   def buildCommandSeq(): Seq[String] = {
-    val command = jobDesc.command
-    val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run";
+    val command = appDesc.command
+    val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run"
     val runScript = new File(sparkHome, script).getCanonicalPath
-    Seq(runScript, command.mainClass) ++ command.arguments.map(substituteVariables)
+    Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(appId)).map(substituteVariables)
   }
 
   /** Spawn a thread that will redirect a given stream to a file */
@@ -96,30 +96,24 @@ private[spark] class ExecutorRunner(
   }
 
   /**
-   * Download and run the executor described in our JobDescription
+   * Download and run the executor described in our ApplicationDescription
    */
   def fetchAndRunExecutor() {
     try {
       // Create the executor's working directory
-      val executorDir = new File(workDir, jobId + "/" + execId)
+      val executorDir = new File(workDir, appId + "/" + execId)
       if (!executorDir.mkdirs()) {
         throw new IOException("Failed to create directory " + executorDir)
       }
 
-      // Download the files it depends on into it (disabled for now)
-      //for (url <- jobDesc.fileUrls) {
-      //  fetchFile(url, executorDir)
-      //}
-
       // Launch the process
       val command = buildCommandSeq()
       val builder = new ProcessBuilder(command: _*).directory(executorDir)
       val env = builder.environment()
-      for ((key, value) <- jobDesc.command.environment) {
+      for ((key, value) <- appDesc.command.environment) {
         env.put(key, value)
       }
-      env.put("SPARK_CORES", cores.toString)
-      env.put("SPARK_MEMORY", memory.toString)
+      env.put("SPARK_MEM", memory.toString + "m")
       // In case we are running this from within the Spark Shell, avoid creating a "scala"
       // parent process for the executor command
       env.put("SPARK_LAUNCH_WITH_SCALA", "0")
@@ -134,7 +128,7 @@ private[spark] class ExecutorRunner(
       // times on the same machine.
       val exitCode = process.waitFor()
       val message = "Command exited with code " + exitCode
-      worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message),
+      worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message),
                                     Some(exitCode))
     } catch {
       case interrupted: InterruptedException =>
@@ -146,7 +140,7 @@ private[spark] class ExecutorRunner(
           process.destroy()
         }
         val message = e.getClass + ": " + e.getMessage
-        worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message), None)
+        worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), None)
       }
     }
   }
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala
index ec25a19e7b9cb5a86d6ace874a7410496820c655..700d87b1c1da2e1a8541ad5c7d729f0392e138fb 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/spark/deploy/worker/Worker.scala
@@ -1,19 +1,18 @@
 package spark.deploy.worker
 
 import scala.collection.mutable.{ArrayBuffer, HashMap}
-import akka.actor.{ActorRef, Props, Actor}
+import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+import akka.util.duration._
 import spark.{Logging, Utils}
 import spark.util.AkkaUtils
 import spark.deploy._
-import akka.remote.RemoteClientLifeCycleEvent
+import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
 import java.text.SimpleDateFormat
 import java.util.Date
-import akka.remote.RemoteClientShutdown
-import akka.remote.RemoteClientDisconnected
 import spark.deploy.RegisterWorker
 import spark.deploy.LaunchExecutor
 import spark.deploy.RegisterWorkerFailed
-import akka.actor.Terminated
+import spark.deploy.master.Master
 import java.io.File
 
 private[spark] class Worker(
@@ -27,7 +26,9 @@ private[spark] class Worker(
   extends Actor with Logging {
 
   val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss")  // For worker and executor IDs
-  val MASTER_REGEX = "spark://([^:]+):([0-9]+)".r
+
+  // Send a heartbeat every (heartbeat timeout) / 4 milliseconds
+  val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4
 
   var master: ActorRef = null
   var masterWebUiUrl : String = ""
@@ -48,11 +49,7 @@ private[spark] class Worker(
   def memoryFree: Int = memory - memoryUsed
 
   def createWorkDir() {
-    workDir = if (workDirPath != null) {
-      new File(workDirPath)
-    } else {
-      new File(sparkHome, "work")
-    }
+    workDir = Option(workDirPath).map(new File(_)).getOrElse(new File(sparkHome, "work"))
     try {
       if (!workDir.exists() && !workDir.mkdirs()) {
         logError("Failed to create work directory " + workDir)
@@ -68,8 +65,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)))
-    val envVar = System.getenv("SPARK_HOME")
-    sparkHome = new File(if (envVar == null) "." else envVar)
+    sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse("."))
     logInfo("Spark home: " + sparkHome)
     createWorkDir()
     connectToMaster()
@@ -77,30 +73,15 @@ private[spark] class Worker(
   }
 
   def connectToMaster() {
-    masterUrl match {
-      case MASTER_REGEX(masterHost, masterPort) => {
-        logInfo("Connecting to master spark://" + masterHost + ":" + masterPort)
-        val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort)
-        try {
-          master = context.actorFor(akkaUrl)
-          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)
-        }
-      }
-
-      case _ =>
-        logError("Invalid master URL: " + masterUrl)
-        System.exit(1)
-    }
+    logInfo("Connecting to master " + masterUrl)
+    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
   }
 
   def startWebUi() {
-    val webUi = new WorkerWebUI(self)
+    val webUi = new WorkerWebUI(context.system, self, workDir)
     try {
       AkkaUtils.startSprayServer(context.system, "0.0.0.0", webUiPort, webUi.handler)
     } catch {
@@ -114,45 +95,54 @@ private[spark] class Worker(
     case RegisteredWorker(url) =>
       masterWebUiUrl = url
       logInfo("Successfully registered with master")
+      context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) {
+        master ! Heartbeat(workerId)
+      }
 
     case RegisterWorkerFailed(message) =>
       logError("Worker registration failed: " + message)
       System.exit(1)
 
-    case LaunchExecutor(jobId, execId, jobDesc, cores_, memory_) =>
-      logInfo("Asked to launch executor %s/%d for %s".format(jobId, execId, jobDesc.name))
+    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(
-        jobId, execId, jobDesc, cores_, memory_, self, workerId, ip, sparkHome, workDir)
-      executors(jobId + "/" + execId) = manager
+        appId, execId, appDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir)
+      executors(appId + "/" + execId) = manager
       manager.start()
       coresUsed += cores_
       memoryUsed += memory_
-      master ! ExecutorStateChanged(jobId, execId, ExecutorState.RUNNING, None, None)
+      master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None)
 
-    case ExecutorStateChanged(jobId, execId, state, message, exitStatus) =>
-      master ! ExecutorStateChanged(jobId, execId, state, message, exitStatus)
-      val fullId = jobId + "/" + execId
+    case ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
+      master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+      val fullId = appId + "/" + execId
       if (ExecutorState.isFinished(state)) {
         val executor = executors(fullId)
-        logInfo("Executor " + fullId + " finished with state " + state)
+        logInfo("Executor " + fullId + " finished with state " + state +
+          message.map(" message " + _).getOrElse("") +
+          exitStatus.map(" exitStatus " + _).getOrElse(""))
         finishedExecutors(fullId) = executor
         executors -= fullId
         coresUsed -= executor.cores
         memoryUsed -= executor.memory
       }
 
-    case KillExecutor(jobId, execId) =>
-      val fullId = jobId + "/" + execId
-      val executor = executors(fullId)
-      logInfo("Asked to kill executor " + fullId)
-      executor.kill()
+    case KillExecutor(appId, execId) =>
+      val fullId = appId + "/" + execId
+      executors.get(fullId) match {
+        case Some(executor) =>
+          logInfo("Asked to kill executor " + fullId)
+          executor.kill()
+        case None =>
+          logInfo("Asked to kill unknown executor " + fullId)
+      }
 
     case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
       masterDisconnected()
-      
+
     case RequestWorkerState => {
-      sender ! WorkerState(ip + ":" + port, workerId, executors.values.toList, 
-        finishedExecutors.values.toList, masterUrl, cores, memory, 
+      sender ! WorkerState(ip, port, workerId, executors.values.toList,
+        finishedExecutors.values.toList, masterUrl, cores, memory,
         coresUsed, memoryUsed, masterWebUiUrl)
     }
   }
@@ -177,11 +167,19 @@ private[spark] class Worker(
 private[spark] object Worker {
   def main(argStrings: Array[String]) {
     val args = new WorkerArguments(argStrings)
-    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port)
-    val actor = actorSystem.actorOf(
-      Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory,
-        args.master, args.workDir)),
-      name = "Worker")
+    val (actorSystem, _) = startSystemAndActor(args.ip, args.port, args.webUiPort, args.cores,
+      args.memory, args.master, args.workDir)
     actorSystem.awaitTermination()
   }
+
+  def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
+    masterUrl: String, workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = {
+    // The LocalSparkCluster runs multiple local sparkWorkerX actor systems
+    val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("")
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
+    val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory,
+      masterUrl, workDir)), name = "Worker")
+    (actorSystem, boundPort)
+  }
+
 }
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
index 37524a7c82f8b347d7fbe5f571a9647ff366bb68..08f02bad80d7f47b2f019745216538eda2640223 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
@@ -92,7 +92,7 @@ private[spark] class WorkerArguments(args: Array[String]) {
       "Options:\n" +
       "  -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 jobs in (default: SPARK_HOME/work)\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" +
       "  -p PORT, --port PORT     Port to listen on (default: random)\n" +
       "  --webui-port PORT        Port for web UI (default: 8081)")
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
index 7dd1781900ea873b45ce257b09fc793df04829ea..99c3b506fa4634c74a349f73a0e37c615a545fe0 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
@@ -12,16 +12,17 @@ import spray.http.MediaTypes._
 
 import spark.deploy.{WorkerState, RequestWorkerState}
 import spark.deploy.JsonProtocol._
+import java.io.File
 
+/**
+ * Web UI server for the standalone worker.
+ */
 private[spark]
-class WorkerWebUI(worker: ActorRef)(implicit val context: ActorContext) extends Directives {
-  import context.dispatcher
-
-  val actorSystem         = context.system
-  val RESOURCE_DIR        = "spark/deploy/worker/webui"
+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(1 seconds)
+  implicit val timeout = Timeout(10 seconds)
 
   val handler = {
     get {
@@ -43,7 +44,7 @@ class WorkerWebUI(worker: ActorRef)(implicit val context: ActorContext) extends
       path("log") {
         parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) =>
           respondWithMediaType(`text/plain`) {
-            getFromFile("work/" + jobId + "/" + executorId + "/" + logType)
+            getFromFileName(workDir.getPath() + "/" + appId + "/" + executorId + "/" + logType)
           }
         }
       } ~
@@ -53,5 +54,4 @@ class WorkerWebUI(worker: ActorRef)(implicit val context: ActorContext) extends
       getFromResourceDirectory(RESOURCE_DIR)
     }
   }
-  
 }
diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala
index 2552958d27e37a0aee8afddba93e40c2fb83f4bd..3e7407b58d8e6dacc52e405f1407c5b713087667 100644
--- a/core/src/main/scala/spark/executor/Executor.scala
+++ b/core/src/main/scala/spark/executor/Executor.scala
@@ -16,61 +16,61 @@ 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(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)
+  // 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)
-    }
+  // Set spark.* system properties from executor arg
+  for ((key, value) <- properties) {
+    System.setProperty(key, value)
+  }
 
-    // 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)
+  // Create our ClassLoader and set it on this thread
+  private val 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)
             }
-          } catch {
-            case oom: OutOfMemoryError => System.exit(ExecutorExitCode.OOM)
-            case t: Throwable => System.exit(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(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))
@@ -80,6 +80,7 @@ private[spark] class Executor extends Logging {
     extends Runnable {
 
     override def run() {
+      val startTime = System.currentTimeMillis()
       SparkEnv.set(env)
       Thread.currentThread.setContextClassLoader(urlClassLoader)
       val ser = SparkEnv.get.closureSerializer.newInstance()
@@ -93,9 +94,18 @@ private[spark] class Executor extends Logging {
         val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
         logInfo("Its generation is " + task.generation)
         env.mapOutputTracker.updateGeneration(task.generation)
+        val taskStart = System.currentTimeMillis()
         val value = task.run(taskId.toInt)
+        val taskFinish = System.currentTimeMillis()
+        task.metrics.foreach{ m =>
+          m.executorDeserializeTime = (taskStart - startTime).toInt
+          m.executorRunTime = (taskFinish - taskStart).toInt
+        }
+        //TODO I'd also like to track the time it takes to serialize the task results, but that is huge headache, b/c
+        // we need to serialize the task metrics first.  If TaskMetrics had a custom serialized format, we could
+        // just change the relevants bytes in the byte buffer
         val accumUpdates = Accumulators.values
-        val result = new TaskResult(value, accumUpdates)
+        val result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null))
         val serializedResult = ser.serialize(result)
         logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit)
         context.statusUpdate(taskId, TaskState.FINISHED, serializedResult)
@@ -159,22 +169,24 @@ private[spark] class Executor extends Logging {
    * SparkContext. Also adds any new JARs we fetched to the class loader.
    */
   private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) {
-    // Fetch missing dependencies
-    for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
-      logInfo("Fetching " + name + " with timestamp " + timestamp)
-      Utils.fetchFile(name, new File("."))
-      currentFiles(name) = timestamp
-    }
-    for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
-      logInfo("Fetching " + name + " with timestamp " + timestamp)
-      Utils.fetchFile(name, new File("."))
-      currentJars(name) = timestamp
-      // Add it to our class loader
-      val localName = name.split("/").last
-      val url = new File(".", localName).toURI.toURL
-      if (!urlClassLoader.getURLs.contains(url)) {
-        logInfo("Adding " + url + " to class loader")
-        urlClassLoader.addURL(url)
+    synchronized {
+      // Fetch missing dependencies
+      for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
+        logInfo("Fetching " + name + " with timestamp " + timestamp)
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
+        currentFiles(name) = timestamp
+      }
+      for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
+        logInfo("Fetching " + name + " with timestamp " + timestamp)
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
+        currentJars(name) = timestamp
+        // Add it to our class loader
+        val localName = name.split("/").last
+        val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL
+        if (!urlClassLoader.getURLs.contains(url)) {
+          logInfo("Adding " + url + " to class loader")
+          urlClassLoader.addURL(url)
+        }
       }
     }
   }
diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
index eeab3959c662896c1148ab06bec7519d27040ffd..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) {
@@ -29,14 +30,22 @@ private[spark] class MesosExecutorBackend(executor: Executor)
       executorInfo: ExecutorInfo,
       frameworkInfo: FrameworkInfo,
       slaveInfo: SlaveInfo) {
+    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(slaveInfo.getHostname, properties)
+    executor = new Executor(
+      executorInfo.getExecutorId.getValue,
+      slaveInfo.getHostname,
+      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) {
@@ -63,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 a29bf974d247ec2ed2445b2e4845bf35d78b18d5..1047f71c6ae0dc13eaf873049fbfeceaa4d1ff69 100644
--- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
@@ -4,75 +4,78 @@ import java.nio.ByteBuffer
 import spark.Logging
 import spark.TaskState.TaskState
 import spark.util.AkkaUtils
-import akka.actor.{ActorRef, Actor, Props}
+import akka.actor.{ActorRef, Actor, Props, Terminated}
+import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
 import java.util.concurrent.{TimeUnit, ThreadPoolExecutor, SynchronousQueue}
-import akka.remote.RemoteClientLifeCycleEvent
 import spark.scheduler.cluster._
-import spark.scheduler.cluster.RegisteredSlave
+import spark.scheduler.cluster.RegisteredExecutor
 import spark.scheduler.cluster.LaunchTask
-import spark.scheduler.cluster.RegisterSlaveFailed
-import spark.scheduler.cluster.RegisterSlave
-
+import spark.scheduler.cluster.RegisterExecutorFailed
+import spark.scheduler.cluster.RegisterExecutor
 
 private[spark] class StandaloneExecutorBackend(
-    executor: Executor,
-    masterUrl: String,
-    slaveId: String,
+    driverUrl: String,
+    executorId: String,
     hostname: String,
     cores: Int)
   extends Actor
   with ExecutorBackend
   with Logging {
 
-  var master: ActorRef = null
+  var executor: Executor = null
+  var driver: ActorRef = null
 
   override def preStart() {
-    try {
-      logInfo("Connecting to master: " + masterUrl)
-      master = context.actorFor(masterUrl)
-      master ! RegisterSlave(slaveId, hostname, cores)
-      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)
-    }
+    logInfo("Connecting to driver: " + driverUrl)
+    driver = context.actorFor(driverUrl)
+    driver ! RegisterExecutor(executorId, hostname, cores)
+    context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+    context.watch(driver) // Doesn't work with remote actors, but useful for testing
   }
 
   override def receive = {
-    case RegisteredSlave(sparkProperties) =>
-      logInfo("Successfully registered with master")
-      executor.initialize(hostname, sparkProperties)
+    case RegisteredExecutor(sparkProperties) =>
+      logInfo("Successfully registered with driver")
+      executor = new Executor(executorId, hostname, sparkProperties)
 
-    case RegisterSlaveFailed(message) =>
+    case RegisterExecutorFailed(message) =>
       logError("Slave registration failed: " + message)
       System.exit(1)
 
     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.")
+      System.exit(1)
   }
 
   override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) {
-    master ! StatusUpdate(slaveId, taskId, state, data)
+    driver ! StatusUpdate(executorId, taskId, state, data)
   }
 }
 
 private[spark] object StandaloneExecutorBackend {
-  def run(masterUrl: String, slaveId: String, hostname: String, cores: Int) {
+  def run(driverUrl: String, executorId: String, hostname: String, cores: Int) {
     // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor
     // before getting started with all our system properties, etc
     val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0)
     val actor = actorSystem.actorOf(
-      Props(new StandaloneExecutorBackend(new Executor, masterUrl, slaveId, hostname, cores)),
+      Props(new StandaloneExecutorBackend(driverUrl, executorId, hostname, cores)),
       name = "Executor")
     actorSystem.awaitTermination()
   }
 
   def main(args: Array[String]) {
-    if (args.length != 4) {
-      System.err.println("Usage: StandaloneExecutorBackend <master> <slaveId> <hostname> <cores>")
+    if (args.length < 4) {
+      //the reason we allow the last frameworkId argument is to make it easy to kill rogue executors
+      System.err.println("Usage: StandaloneExecutorBackend <driverUrl> <executorId> <hostname> <cores> [<appid>]")
       System.exit(1)
     }
     run(args(0), args(1), args(2), args(3).toInt)
diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala
new file mode 100644
index 0000000000000000000000000000000000000000..93bbb6b4587a5b6773ad9597e0b767c0e0531688
--- /dev/null
+++ b/core/src/main/scala/spark/executor/TaskMetrics.scala
@@ -0,0 +1,78 @@
+package spark.executor
+
+class TaskMetrics extends Serializable {
+  /**
+   * Time taken on the executor to deserialize this task
+   */
+  var executorDeserializeTime: Int = _
+
+  /**
+   * Time the executor spends actually running the task (including fetching shuffle data)
+   */
+  var executorRunTime:Int = _
+
+  /**
+   * The number of bytes this task transmitted back to the driver as the TaskResult
+   */
+  var resultSize: Long = _
+
+  /**
+   * If this task reads from shuffle output, metrics on getting shuffle data will be collected here
+   */
+  var shuffleReadMetrics: Option[ShuffleReadMetrics] = None
+
+  /**
+   * If this task writes to shuffle output, metrics on the written shuffle data will be collected here
+   */
+  var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None
+}
+
+object TaskMetrics {
+  private[spark] def empty(): TaskMetrics = new TaskMetrics
+}
+
+
+class ShuffleReadMetrics extends Serializable {
+  /**
+   * Total number of blocks fetched in a shuffle (remote or local)
+   */
+  var totalBlocksFetched : Int = _
+
+  /**
+   * Number of remote blocks fetched in a shuffle
+   */
+  var remoteBlocksFetched: Int = _
+
+  /**
+   * Local blocks fetched in a shuffle
+   */
+  var localBlocksFetched: Int = _
+
+  /**
+   * Total time to read shuffle data
+   */
+  var shuffleReadMillis: Long = _
+
+  /**
+   * Total time that is spent blocked waiting for shuffle to fetch data
+   */
+  var fetchWaitTime: Long = _
+
+  /**
+   * The total amount of time for all the shuffle fetches.  This adds up time from overlapping
+   *     shuffles, so can be longer than task time
+   */
+  var remoteFetchTime: Long = _
+
+  /**
+   * Total number of remote bytes read from a shuffle
+   */
+  var remoteBytesRead: Long = _
+}
+
+class ShuffleWriteMetrics extends Serializable {
+  /**
+   * Number of bytes written for a shuffle
+   */
+  var shuffleBytesWritten: Long = _
+}
diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala
index c193bf7c8d37ed748b7d40ca8c58565410abdd87..d1451bc2124c581eff01dfae5277612ea5c995c7 100644
--- a/core/src/main/scala/spark/network/Connection.scala
+++ b/core/src/main/scala/spark/network/Connection.scala
@@ -12,7 +12,14 @@ import java.net._
 
 
 private[spark]
-abstract class Connection(val channel: SocketChannel, val selector: Selector) extends Logging {
+abstract class Connection(val channel: SocketChannel, val selector: Selector,
+                          val remoteConnectionManagerId: ConnectionManagerId) extends Logging {
+  def this(channel_ : SocketChannel, selector_ : Selector) = {
+    this(channel_, selector_,
+         ConnectionManagerId.fromSocketAddress(
+            channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]
+         ))
+  }
 
   channel.configureBlocking(false)
   channel.socket.setTcpNoDelay(true)
@@ -25,7 +32,6 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex
   var onKeyInterestChangeCallback: (Connection, Int) => Unit = null
 
   val remoteAddress = getRemoteAddress()
-  val remoteConnectionManagerId = ConnectionManagerId.fromSocketAddress(remoteAddress)
 
   def key() = channel.keyFor(selector)
 
@@ -103,8 +109,9 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex
 }
 
 
-private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector) 
-extends Connection(SocketChannel.open, selector_) {
+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]()
@@ -191,7 +198,7 @@ extends Connection(SocketChannel.open, selector_) {
     outbox.synchronized {
       outbox.addMessage(message)
       if (channel.isConnected) {
-        changeConnectionKeyInterest(SelectionKey.OP_WRITE)
+        changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
       }
     }
   }
@@ -212,7 +219,7 @@ extends Connection(SocketChannel.open, selector_) {
   def finishConnect() {
     try {
       channel.finishConnect
-      changeConnectionKeyInterest(SelectionKey.OP_WRITE)
+      changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
       logInfo("Connected to [" + address + "], " + outbox.messages.size + " messages pending")
     } catch {
       case e: Exception => {
@@ -232,8 +239,7 @@ extends Connection(SocketChannel.open, selector_) {
                 currentBuffers ++= chunk.buffers 
               }
               case None => {
-                changeConnectionKeyInterest(0)
-                /*key.interestOps(0)*/
+                changeConnectionKeyInterest(SelectionKey.OP_READ)
                 return
               }
             }
@@ -260,6 +266,23 @@ extends Connection(SocketChannel.open, selector_) {
       }
     }
   }
+
+  override def read() {
+    // 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)
+      }
+    } catch {
+      case e: Exception =>
+        logError("Exception while reading SendingConnection to " + remoteConnectionManagerId, e)
+        callOnExceptionCallback(e)
+        close()
+    }
+  }
 }
 
 
diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala
index 04b303afe07ceaafb41c77e5d0b95ff728381a8b..8f8892b8c778f374a807bb1f0a4835e8e0501f79 100644
--- a/core/src/main/scala/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/spark/network/ConnectionManager.scala
@@ -52,9 +52,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
   val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)]
   val sendMessageRequests = new Queue[(Message, SendingConnection)]
 
-  implicit val futureExecContext = ExecutionContext.fromExecutor(
-    Executors.newCachedThreadPool(DaemonThreadFactory))
-  
+  implicit val futureExecContext = ExecutionContext.fromExecutor(Utils.newDaemonCachedThreadPool())
+
   var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null
 
   serverChannel.configureBlocking(false)
@@ -67,31 +66,28 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
   val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort)
   logInfo("Bound socket to port " + serverChannel.socket.getLocalPort() + " with id = " + id)
   
-  val thisInstance = this
   val selectorThread = new Thread("connection-manager-thread") {
-    override def run() {
-      thisInstance.run()
-    }
+    override def run() = ConnectionManager.this.run()
   }
   selectorThread.setDaemon(true)
   selectorThread.start()
 
-  def run() {
+  private def run() {
     try {
       while(!selectorThread.isInterrupted) {
-        for( (connectionManagerId, sendingConnection) <- connectionRequests) {
+        for ((connectionManagerId, sendingConnection) <- connectionRequests) {
           sendingConnection.connect() 
           addConnection(sendingConnection)
           connectionRequests -= connectionManagerId
         }
         sendMessageRequests.synchronized {
-          while(!sendMessageRequests.isEmpty) {
+          while (!sendMessageRequests.isEmpty) {
             val (message, connection) = sendMessageRequests.dequeue
             connection.send(message)
           }
         }
 
-        while(!keyInterestChangeRequests.isEmpty) {
+        while (!keyInterestChangeRequests.isEmpty) {
           val (key, ops) = keyInterestChangeRequests.dequeue
           val connection = connectionsByKey(key)
           val lastOps = key.interestOps()
@@ -127,14 +123,11 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
           if (key.isValid) {
             if (key.isAcceptable) {
               acceptConnection(key)
-            } else 
-            if (key.isConnectable) {
+            } else if (key.isConnectable) {
               connectionsByKey(key).asInstanceOf[SendingConnection].finishConnect()
-            } else 
-            if (key.isReadable) {
+            } else if (key.isReadable) {
               connectionsByKey(key).read()
-            } else 
-            if (key.isWritable) {
+            } else if (key.isWritable) {
               connectionsByKey(key).write()
             }
           }
@@ -145,7 +138,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
     }
   }
   
-  def acceptConnection(key: SelectionKey) {
+  private def acceptConnection(key: SelectionKey) {
     val serverChannel = key.channel.asInstanceOf[ServerSocketChannel]
     val newChannel = serverChannel.accept()
     val newConnection = new ReceivingConnection(newChannel, selector)
@@ -155,7 +148,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
     logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]")
   }
 
-  def addConnection(connection: Connection) {
+  private def addConnection(connection: Connection) {
     connectionsByKey += ((connection.key, connection))
     if (connection.isInstanceOf[SendingConnection]) {
       val sendingConnection = connection.asInstanceOf[SendingConnection]
@@ -166,7 +159,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
     connection.onClose(removeConnection)
   }
 
-  def removeConnection(connection: Connection) {
+  private def removeConnection(connection: Connection) {
     connectionsByKey -= connection.key
     if (connection.isInstanceOf[SendingConnection]) {
       val sendingConnection = connection.asInstanceOf[SendingConnection]
@@ -223,16 +216,16 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
     }
   }
 
-  def handleConnectionError(connection: Connection, e: Exception) {
+  private def handleConnectionError(connection: Connection, e: Exception) {
     logInfo("Handling connection error on connection to " + connection.remoteConnectionManagerId)
     removeConnection(connection)
   }
 
-  def changeConnectionKeyInterest(connection: Connection, ops: Int) {
+  private def changeConnectionKeyInterest(connection: Connection, ops: Int) {
     keyInterestChangeRequests += ((connection.key, ops))  
   }
 
-  def receiveMessage(connection: Connection, message: Message) {
+  private def receiveMessage(connection: Connection, message: Message) {
     val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress)
     logDebug("Received [" + message + "] from [" + connectionManagerId + "]") 
     val runnable = new Runnable() {
@@ -300,7 +293,8 @@ 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))
+      val newConnection = connectionRequests.getOrElseUpdate(connectionManagerId,
+          new SendingConnection(inetSocketAddress, selector, connectionManagerId))
       newConnection   
     }
     val lookupKey = ConnectionManagerId.fromSocketAddress(connectionManagerId.toSocketAddress)
@@ -351,7 +345,6 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
 private[spark] object ConnectionManager {
 
   def main(args: Array[String]) {
-  
     val manager = new ConnectionManager(9999)
     manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { 
       println("Received [" + msg + "] from [" + id + "]")
diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
index 42f46e06ed0f4dabe06e149de758965d029e5d83..de2dce161a0906f727f563674220f061dcb6297d 100644
--- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala
+++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
@@ -20,7 +20,7 @@ private[spark] class ApproximateActionListener[T, U, R](
   extends JobListener {
 
   val startTime = System.currentTimeMillis()
-  val totalTasks = rdd.splits.size
+  val totalTasks = rdd.partitions.size
   var finishedTasks = 0
   var failure: Option[Exception] = None             // Set if the job has failed (permanently)
   var resultObject: Option[PartialResult[R]] = None // Set if we've already returned a PartialResult
@@ -32,7 +32,7 @@ private[spark] class ApproximateActionListener[T, U, R](
       if (finishedTasks == totalTasks) {
         // If we had already returned a PartialResult, set its final value
         resultObject.foreach(r => r.setFinalValue(evaluator.currentResult()))
-        // Notify any waiting thread that may have called getResult
+        // Notify any waiting thread that may have called awaitResult
         this.notifyAll()
       }
     }
@@ -49,7 +49,7 @@ private[spark] class ApproximateActionListener[T, U, R](
    * Waits for up to timeout milliseconds since the listener was created and then returns a
    * PartialResult with the result so far. This may be complete if the whole job is done.
    */
-  def getResult(): PartialResult[R] = synchronized {
+  def awaitResult(): PartialResult[R] = synchronized {
     val finishTime = startTime + timeout
     while (true) {
       val time = System.currentTimeMillis()
diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala
index f98528a183e6bb2d295bef699658290745e871fe..7348c4f15bad60a94e736a2e10cec2a018233e99 100644
--- a/core/src/main/scala/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/spark/rdd/BlockRDD.scala
@@ -1,36 +1,31 @@
 package spark.rdd
 
 import scala.collection.mutable.HashMap
+import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
 
-import spark.{Dependency, RDD, SparkContext, SparkEnv, Split, TaskContext}
-
-
-private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
+private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition {
   val index = idx
 }
 
 private[spark]
 class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
-  extends RDD[T](sc) {
-
-  @transient
-  val splits_ = (0 until blockIds.size).map(i => {
-    new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split]
-  }).toArray
+  extends RDD[T](sc, Nil) {
 
-  @transient
-  lazy val locations_  = {
+  @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):_*)
   }
 
-  override def splits = splits_
+  override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => {
+    new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition]
+  }).toArray
+
 
-  override def compute(split: Split, context: TaskContext): Iterator[T] = {
+  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
     val blockManager = SparkEnv.get.blockManager
-    val blockId = split.asInstanceOf[BlockRDDSplit].blockId
+    val blockId = split.asInstanceOf[BlockRDDPartition].blockId
     blockManager.get(blockId) match {
       case Some(block) => block.asInstanceOf[Iterator[T]]
       case None =>
@@ -38,9 +33,8 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
     }
   }
 
-  override def preferredLocations(split: Split) =
-    locations_(split.asInstanceOf[BlockRDDSplit].blockId)
+  override def getPreferredLocations(split: Partition): Seq[String] =
+    locations_(split.asInstanceOf[BlockRDDPartition].blockId)
 
-  override val dependencies: List[Dependency[_]] = Nil
 }
 
diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala
index 4a7e5f3d0602887a4b9f26dc0db170a9f038d40e..38600b8be4e544206ef4aedebfcb79d72efdb187 100644
--- a/core/src/main/scala/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala
@@ -1,53 +1,73 @@
 package spark.rdd
 
-import spark.{NarrowDependency, RDD, SparkContext, Split, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
+import spark._
 
 
 private[spark]
-class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable {
+class CartesianPartition(
+    idx: Int,
+    @transient rdd1: RDD[_],
+    @transient rdd2: RDD[_],
+    s1Index: Int,
+    s2Index: Int
+  ) extends Partition {
+  var s1 = rdd1.partitions(s1Index)
+  var s2 = rdd2.partitions(s2Index)
   override val index: Int = idx
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent split at the time of task serialization
+    s1 = rdd1.partitions(s1Index)
+    s2 = rdd2.partitions(s2Index)
+    oos.defaultWriteObject()
+  }
 }
 
 private[spark]
 class CartesianRDD[T: ClassManifest, U:ClassManifest](
     sc: SparkContext,
-    rdd1: RDD[T],
-    rdd2: RDD[U])
-  extends RDD[Pair[T, U]](sc)
+    var rdd1 : RDD[T],
+    var rdd2 : RDD[U])
+  extends RDD[Pair[T, U]](sc, Nil)
   with Serializable {
 
-  val numSplitsInRdd2 = rdd2.splits.size
+  val numPartitionsInRdd2 = rdd2.partitions.size
 
-  @transient
-  val splits_ = {
+  override def getPartitions: Array[Partition] = {
     // create the cross product split
-    val array = new Array[Split](rdd1.splits.size * rdd2.splits.size)
-    for (s1 <- rdd1.splits; s2 <- rdd2.splits) {
-      val idx = s1.index * numSplitsInRdd2 + s2.index
-      array(idx) = new CartesianSplit(idx, s1, s2)
+    val array = new Array[Partition](rdd1.partitions.size * rdd2.partitions.size)
+    for (s1 <- rdd1.partitions; s2 <- rdd2.partitions) {
+      val idx = s1.index * numPartitionsInRdd2 + s2.index
+      array(idx) = new CartesianPartition(idx, rdd1, rdd2, s1.index, s2.index)
     }
     array
   }
 
-  override def splits = splits_
-
-  override def preferredLocations(split: Split) = {
-    val currSplit = split.asInstanceOf[CartesianSplit]
+  override def getPreferredLocations(split: Partition): Seq[String] = {
+    val currSplit = split.asInstanceOf[CartesianPartition]
     rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)
   }
 
-  override def compute(split: Split, context: TaskContext) = {
-    val currSplit = split.asInstanceOf[CartesianSplit]
+  override def compute(split: Partition, context: TaskContext) = {
+    val currSplit = split.asInstanceOf[CartesianPartition]
     for (x <- rdd1.iterator(currSplit.s1, context);
       y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
   }
 
-  override val dependencies = List(
+  override def getDependencies: Seq[Dependency[_]] = List(
     new NarrowDependency(rdd1) {
-      def getParents(id: Int): Seq[Int] = List(id / numSplitsInRdd2)
+      def getParents(id: Int): Seq[Int] = List(id / numPartitionsInRdd2)
     },
     new NarrowDependency(rdd2) {
-      def getParents(id: Int): Seq[Int] = List(id % numSplitsInRdd2)
+      def getParents(id: Int): Seq[Int] = List(id % numPartitionsInRdd2)
     }
   )
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+  }
 }
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5db77eb142453b70129117fee9ba1ce671a9f19c
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -0,0 +1,127 @@
+package spark.rdd
+
+import spark._
+import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.io.{NullWritable, BytesWritable}
+import org.apache.hadoop.util.ReflectionUtils
+import org.apache.hadoop.fs.Path
+import java.io.{File, IOException, EOFException}
+import java.text.NumberFormat
+
+private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
+
+/**
+ * This RDD represents a RDD checkpoint file (similar to HadoopRDD).
+ */
+private[spark]
+class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: String)
+  extends RDD[T](sc, Nil) {
+
+  @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)
+    }
+    Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i))
+  }
+
+  checkpointData = Some(new RDDCheckpointData[T](this))
+  checkpointData.get.cpFile = Some(checkpointPath)
+
+  override def getPreferredLocations(split: Partition): Seq[String] = {
+    val status = fs.getFileStatus(new Path(checkpointPath))
+    val locations = fs.getFileBlockLocations(status, 0, status.getLen)
+    locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost")
+  }
+
+  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+    val file = new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))
+    CheckpointRDD.readFromFile(file, context)
+  }
+
+  override def checkpoint() {
+    // Do nothing. CheckpointRDD should not be checkpointed.
+  }
+}
+
+private[spark] object CheckpointRDD extends Logging {
+
+  def splitIdToFile(splitId: Int): String = {
+    "part-%05d".format(splitId)
+  }
+
+  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 finalOutputName = splitIdToFile(ctx.splitId)
+    val finalOutputPath = new Path(outputDir, finalOutputName)
+    val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId)
+
+    if (fs.exists(tempOutputPath)) {
+      throw new IOException("Checkpoint failed: temporary path " +
+        tempOutputPath + " already exists")
+    }
+    val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+
+    val fileOutputStream = if (blockSize < 0) {
+      fs.create(tempOutputPath, false, bufferSize)
+    } else {
+      // This is mainly for testing purpose
+      fs.create(tempOutputPath, false, bufferSize, fs.getDefaultReplication, blockSize)
+    }
+    val serializer = SparkEnv.get.serializer.newInstance()
+    val serializeStream = serializer.serializeStream(fileOutputStream)
+    serializeStream.writeAll(iterator)
+    serializeStream.close()
+
+    if (!fs.rename(tempOutputPath, finalOutputPath)) {
+      if (!fs.exists(finalOutputPath)) {
+        fs.delete(tempOutputPath, false)
+        throw new IOException("Checkpoint failed: failed to save output of task: "
+          + ctx.attemptId + " and final output path does not exist")
+      } else {
+        // Some other copy of this task must've finished before us and renamed it
+        logInfo("Final output path " + finalOutputPath + " already exists; not overwriting it")
+        fs.delete(tempOutputPath, false)
+      }
+    }
+  }
+
+  def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = {
+    val fs = path.getFileSystem(new Configuration())
+    val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+    val fileInputStream = fs.open(path, bufferSize)
+    val serializer = SparkEnv.get.serializer.newInstance()
+    val deserializeStream = serializer.deserializeStream(fileInputStream)
+
+    // Register an on-task-completion callback to close the input stream.
+    context.addOnCompleteCallback(() => deserializeStream.close())
+
+    deserializeStream.asIterator.asInstanceOf[Iterator[T]]
+  }
+
+  // Test whether CheckpointRDD generate expected number of partitions despite
+  // each split file having multiple blocks. This needs to be run on a
+  // cluster (mesos or standalone) using HDFS.
+  def main(args: Array[String]) {
+    import spark._
+
+    val Array(cluster, hdfsPath) = args
+    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())
+    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, true)
+  }
+}
diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
index ce5f1719117a6c606c1b2d2fcad12594a8ba9907..a6235491cab9657074a6de200d5b4cc3d5321341 100644
--- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
@@ -1,19 +1,36 @@
 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, Split, TaskContext}
+import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext}
 import spark.{Dependency, OneToOneDependency, ShuffleDependency}
 
 
 private[spark] sealed trait CoGroupSplitDep extends Serializable
-private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
+
+private[spark] case class NarrowCoGroupSplitDep(
+    rdd: RDD[_],
+    splitIndex: Int,
+    var split: Partition
+  ) extends CoGroupSplitDep {
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent split at the time of task serialization
+    split = rdd.partitions(splitIndex)
+    oos.defaultWriteObject()
+  }
+}
+
 private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
 
 private[spark]
-class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable {
+class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep])
+  extends Partition with Serializable {
   override val index: Int = idx
   override def hashCode(): Int = idx
 }
@@ -25,54 +42,65 @@ private[spark] class CoGroupAggregator
     { (b1, b2) => b1 ++ b2 })
   with Serializable
 
-class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
-  extends RDD[(K, Seq[Seq[_]])](rdds.head.context) with Logging {
 
-  val aggr = new CoGroupAggregator
+/**
+ * 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)
+  extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) {
 
-  @transient
-  override val dependencies = {
-    val deps = new ArrayBuffer[Dependency[_]]
-    for ((rdd, index) <- rdds.zipWithIndex) {
-      val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
-      if (mapSideCombinedRDD.partitioner == Some(part)) {
-        logInfo("Adding one-to-one dependency with " + mapSideCombinedRDD)
-        deps += new OneToOneDependency(mapSideCombinedRDD)
+  private val aggr = new CoGroupAggregator
+
+  override def getDependencies: Seq[Dependency[_]] = {
+    rdds.map { rdd =>
+      if (rdd.partitioner == Some(part)) {
+        logInfo("Adding one-to-one dependency with " + rdd)
+        new OneToOneDependency(rdd)
       } else {
         logInfo("Adding shuffle dependency with " + rdd)
-        deps += new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part)
+        if (mapSideCombine) {
+          val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
+          new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part)
+        } else {
+          new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part)
+        }
       }
     }
-    deps.toList
   }
 
-  @transient
-  val splits_ : Array[Split] = {
-    val firstRdd = rdds.head
-    val array = new Array[Split](part.numPartitions)
+  override def getPartitions: Array[Partition] = {
+    val array = new Array[Partition](part.numPartitions)
     for (i <- 0 until array.size) {
-      array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (r, j) =>
+      // Each CoGroupPartition will have a dependency per contributing RDD
+      array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) =>
+        // Assume each RDD contributed a single dependency, and get it
         dependencies(j) match {
           case s: ShuffleDependency[_, _] =>
-            new ShuffleCoGroupSplitDep(s.shuffleId): CoGroupSplitDep
+            new ShuffleCoGroupSplitDep(s.shuffleId)
           case _ =>
-            new NarrowCoGroupSplitDep(r, r.splits(i)): CoGroupSplitDep
+            new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))
         }
-      }.toList)
+      }.toArray)
     }
     array
   }
 
-  override def splits = splits_
-
   override val partitioner = Some(part)
 
-  override def preferredLocations(s: Split) = Nil
-
-  override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
-    val split = s.asInstanceOf[CoGroupSplit]
+  override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
+    val split = s.asInstanceOf[CoGroupPartition]
     val numRdds = split.deps.size
+    // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs)
     val map = new JHashMap[K, Seq[ArrayBuffer[Any]]]
+
     def getSeq(k: K): Seq[ArrayBuffer[Any]] = {
       val seq = map.get(k)
       if (seq != null) {
@@ -83,8 +111,9 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
         seq
       }
     }
+
     for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
-      case NarrowCoGroupSplitDep(rdd, itsSplit) => {
+      case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
         // Read them from the parent
         for ((k, v) <- rdd.iterator(itsSplit, context)) {
           getSeq(k.asInstanceOf[K])(depNum) += v
@@ -92,15 +121,25 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
       }
       case ShuffleCoGroupSplitDep(shuffleId) => {
         // Read map outputs of shuffle
-        def mergePair(pair: (K, Seq[Any])) {
-          val mySeq = getSeq(pair._1)
-          for (v <- pair._2)
-            mySeq(depNum) += v
-        }
         val fetcher = SparkEnv.get.shuffleFetcher
-        fetcher.fetch[K, Seq[Any]](shuffleId, split.index).foreach(mergePair)
+        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).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).foreach {
+            case (key, value) => getSeq(key)(depNum) += value
+          }
+        }
       }
     }
     JavaConversions.mapAsScalaMap(map).iterator
   }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdds = null
+  }
 }
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
index 1affe0e0ef817c1276855960d59de574958a50cb..6d862c0c2898a6b94a43ff8085bab6edc685066a 100644
--- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
@@ -1,9 +1,22 @@
 package spark.rdd
 
-import spark.{NarrowDependency, RDD, Split, TaskContext}
+import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
 
+private[spark] case class CoalescedRDDPartition(
+    index: Int,
+    @transient rdd: RDD[_],
+    parentsIndices: Array[Int]
+  ) extends Partition {
+  var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_))
 
-private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent split at the time of task serialization
+    parents = parentsIndices.map(rdd.partitions(_))
+    oos.defaultWriteObject()
+  }
+}
 
 /**
  * Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of
@@ -13,34 +26,39 @@ private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) exten
  * This transformation is useful when an RDD with many partitions gets filtered into a smaller one,
  * or to avoid having a large number of small tasks when processing a directory with many files.
  */
-class CoalescedRDD[T: ClassManifest](prev: RDD[T], maxPartitions: Int)
-  extends RDD[T](prev.context) {
+class CoalescedRDD[T: ClassManifest](
+    @transient var prev: RDD[T],
+    maxPartitions: Int)
+  extends RDD[T](prev.context, Nil) {  // Nil since we implement getDependencies
 
-  @transient val splits_ : Array[Split] = {
-    val prevSplits = prev.splits
+  override def getPartitions: Array[Partition] = {
+    val prevSplits = prev.partitions
     if (prevSplits.length < maxPartitions) {
-      prevSplits.zipWithIndex.map{ case (s, idx) => new CoalescedRDDSplit(idx, Array(s)) }
+      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
-        new CoalescedRDDSplit(i, prevSplits.slice(rangeStart, rangeEnd))
+        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
     }
   }
 
-  override def splits = splits_
-
-  override def compute(split: Split, context: TaskContext): Iterator[T] = {
-    split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap {
-      parentSplit => prev.iterator(parentSplit, context)
+  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+    split.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentSplit =>
+      firstParent[T].iterator(parentSplit, context)
     }
   }
 
-  val dependencies = List(
-    new NarrowDependency(prev) {
+  override def getDependencies: Seq[Dependency[_]] = {
+    Seq(new NarrowDependency(prev) {
       def getParents(id: Int): Seq[Int] =
-        splits(id).asInstanceOf[CoalescedRDDSplit].parents.map(_.index)
-    }
-  )
+        partitions(id).asInstanceOf[CoalescedRDDPartition].parentsIndices
+    })
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    prev = null
+  }
 }
diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala
index d46549b8b6feecf64d3f2dd9063cd28ce7d23c19..c84ec39d21ff7e8f8414a08920a52c285b689d8e 100644
--- a/core/src/main/scala/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala
@@ -1,12 +1,16 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{OneToOneDependency, RDD, Partition, TaskContext}
 
+private[spark] class FilteredRDD[T: ClassManifest](
+    prev: RDD[T],
+    f: T => Boolean)
+  extends RDD[T](prev) {
+
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
 
-private[spark]
-class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) {
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
   override val partitioner = prev.partitioner    // Since filter cannot change a partition's keys
-  override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).filter(f)
+
+  override def compute(split: Partition, context: TaskContext) =
+    firstParent[T].iterator(split, context).filter(f)
 }
diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
index 785662b2da74355fc77caf9b352179f4c34e0a09..8ebc77892514c86d0eec4173daf770ebe5f75b95 100644
--- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
@@ -1,16 +1,16 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
+
 
 private[spark]
 class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
     prev: RDD[T],
     f: T => TraversableOnce[U])
-  extends RDD[U](prev.context) {
+  extends RDD[U](prev) {
 
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
 
-  override def compute(split: Split, context: TaskContext) =
-    prev.iterator(split, context).flatMap(f)
+  override def compute(split: Partition, context: TaskContext) =
+    firstParent[T].iterator(split, context).flatMap(f)
 }
diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala
index fac8ffb4cb4f04752606b28ee1c4936ae116a07e..e16c7ba881977a9f7bd2a593f122d22a596d88c4 100644
--- a/core/src/main/scala/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala
@@ -1,12 +1,12 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
 
+private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
+  extends RDD[Array[T]](prev) {
 
-private[spark]
-class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev.context) {
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override def compute(split: Split, context: TaskContext) =
-    Array(prev.iterator(split, context).toArray).iterator
-}
\ No newline at end of file
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
+
+  override def compute(split: Partition, context: TaskContext) =
+    Array(firstParent[T].iterator(split, context).toArray).iterator
+}
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index ab163f569b656bbfe2b98cbdb70461e97ab44161..cbf5512e247a46773294a428b8f2d6f9d5034876 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -15,16 +15,17 @@ import org.apache.hadoop.mapred.RecordReader
 import org.apache.hadoop.mapred.Reporter
 import org.apache.hadoop.util.ReflectionUtils
 
-import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext}
+import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
+import spark.util.NextIterator
+import org.apache.hadoop.conf.Configurable
 
 
 /**
  * A Spark split class that wraps around a Hadoop InputSplit.
  */
-private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit)
-  extends Split
-  with Serializable {
-
+private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSplit)
+  extends Partition {
+  
   val inputSplit = new SerializableWritable[InputSplit](s)
 
   override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt
@@ -43,18 +44,20 @@ class HadoopRDD[K, V](
     keyClass: Class[K],
     valueClass: Class[V],
     minSplits: Int)
-  extends RDD[(K, V)](sc) {
+  extends RDD[(K, V)](sc, Nil) with Logging {
 
   // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
-  val confBroadcast = sc.broadcast(new SerializableWritable(conf))
+  private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
 
-  @transient
-  val splits_ : Array[Split] = {
+  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[Split](inputSplits.size)
+    val array = new Array[Partition](inputSplits.size)
     for (i <- 0 until inputSplits.size) {
-      array(i) = new HadoopSplit(id, i, inputSplits(i))
+      array(i) = new HadoopPartition(id, i, inputSplits(i))
     }
     array
   }
@@ -64,57 +67,49 @@ class HadoopRDD[K, V](
       .asInstanceOf[InputFormat[K, V]]
   }
 
-  override def splits = splits_
-
-  override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
-    val split = theSplit.asInstanceOf[HadoopSplit]
+  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(() => reader.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
-      }
-      if (finished) {
-        reader.close()
-      }
-      !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)
     }
+
+    override def close() {
+      try {
+        reader.close()
+      } catch {
+        case e: Exception => logWarning("Exception in RecordReader.close()", e)
+      }
+    }
   }
 
-  override def preferredLocations(split: Split) = {
+  override def getPreferredLocations(split: Partition): Seq[String] = {
     // TODO: Filtering out "localhost" in case of file:// URLs
-    val hadoopSplit = split.asInstanceOf[HadoopSplit]
+    val hadoopSplit = split.asInstanceOf[HadoopPartition]
     hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost")
   }
 
-  override val dependencies: List[Dependency[_]] = Nil
+  override def checkpoint() {
+    // Do nothing. Hadoop RDD should not be checkpointed.
+  }
 }
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
index c76450534523c023263d724acf3c51effd9811e3..d283c5b2bb8dfc79c20240476ca9271020fbc480 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
@@ -1,6 +1,6 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
 
 
 private[spark]
@@ -8,11 +8,13 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
     prev: RDD[T],
     f: Iterator[T] => Iterator[U],
     preservesPartitioning: Boolean = false)
-  extends RDD[U](prev.context) {
+  extends RDD[U](prev) {
 
-  override val partitioner = if (preservesPartitioning) prev.partitioner else None
+  override val partitioner =
+    if (preservesPartitioning) firstParent[T].partitioner else None
 
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override def compute(split: Split, context: TaskContext) = f(prev.iterator(split, context))
-}
\ No newline at end of file
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
+
+  override def compute(split: Partition, context: TaskContext) =
+    f(firstParent[T].iterator(split, context))
+}
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..afb7504ba120dbc4a42beef8b001dbfc7cbb722f
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
@@ -0,0 +1,24 @@
+package spark.rdd
+
+import spark.{RDD, Partition, TaskContext}
+
+
+/**
+ * A variant of the MapPartitionsRDD that passes the partition index into the
+ * closure. This can be used to generate or collect partition specific
+ * information such as the number of tuples in a partition.
+ */
+private[spark]
+class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest](
+    prev: RDD[T],
+    f: (Int, Iterator[T]) => Iterator[U],
+    preservesPartitioning: Boolean
+  ) extends RDD[U](prev) {
+
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
+
+  override val partitioner = if (preservesPartitioning) prev.partitioner else None
+
+  override def compute(split: Partition, context: TaskContext) =
+    f(split.index, firstParent[T].iterator(split, context))
+}
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
deleted file mode 100644
index 3d9888bd3477c903dd6bafe2fbe38e5c6976ad7b..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
+++ /dev/null
@@ -1,22 +0,0 @@
-package spark.rdd
-
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
-
-/**
- * A variant of the MapPartitionsRDD that passes the split index into the
- * closure. This can be used to generate or collect partition specific
- * information such as the number of tuples in a partition.
- */
-private[spark]
-class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest](
-    prev: RDD[T],
-    f: (Int, Iterator[T]) => Iterator[U],
-    preservesPartitioning: Boolean)
-  extends RDD[U](prev.context) {
-
-  override val partitioner = if (preservesPartitioning) prev.partitioner else None
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override def compute(split: Split, context: TaskContext) =
-    f(split.index, prev.iterator(split, context))
-}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala
index 70fa8f449789746fab23da242f8f35282edb3e68..af07311b6d0385673150866bb9288e7a270c1da9 100644
--- a/core/src/main/scala/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/MappedRDD.scala
@@ -1,14 +1,13 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
 
 private[spark]
-class MappedRDD[U: ClassManifest, T: ClassManifest](
-    prev: RDD[T],
-    f: T => U)
-  extends RDD[U](prev.context) {
-
-  override def splits = prev.splits
-  override val dependencies = List(new OneToOneDependency(prev))
-  override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).map(f)
-}
\ No newline at end of file
+class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U)
+  extends RDD[U](prev) {
+
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
+
+  override def compute(split: Partition, context: TaskContext) =
+    firstParent[T].iterator(split, context).map(f)
+}
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index 197ed5ea17e2226af5f4aa5f9d4f39fe981d524b..bdd974590af59d7dba4435ae2b94afab3a9a03eb 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -3,16 +3,16 @@ 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._
 
-import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext}
+import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
 
 
 private[spark]
-class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
-  extends Split {
+class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
+  extends Partition {
 
   val serializableHadoopSplit = new SerializableWritable(rawSplit)
 
@@ -20,15 +20,17 @@ class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit
 }
 
 class NewHadoopRDD[K, V](
-    sc: SparkContext,
+    sc : SparkContext,
     inputFormatClass: Class[_ <: InputFormat[K, V]],
-    keyClass: Class[K], valueClass: Class[V],
+    keyClass: Class[K],
+    valueClass: Class[V],
     @transient conf: Configuration)
-  extends RDD[(K, V)](sc)
-  with HadoopMapReduceUtil {
+  extends RDD[(K, V)](sc, Nil)
+  with HadoopMapReduceUtil
+  with Logging {
 
   // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
-  val confBroadcast = sc.broadcast(new SerializableWritable(conf))
+  private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
   // private val serializableConf = new SerializableWritable(conf)
 
   private val jobtrackerId: String = {
@@ -36,35 +38,37 @@ class NewHadoopRDD[K, V](
     formatter.format(new Date())
   }
 
-  @transient
-  private val jobId = new JobID(jobtrackerId, id)
+  @transient private val jobId = new JobID(jobtrackerId, id)
 
-  @transient
-  private val splits_ : Array[Split] = {
+  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[Split](rawSplits.size)
+    val result = new Array[Partition](rawSplits.size)
     for (i <- 0 until rawSplits.size) {
-      result(i) = new NewHadoopSplit(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
+      result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
     }
     result
   }
 
-  override def splits = splits_
-
-  override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
-    val split = theSplit.asInstanceOf[NewHadoopSplit]
+  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 hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
     val format = inputFormatClass.newInstance
+    if (format.isInstanceOf[Configurable]) {
+      format.asInstanceOf[Configurable].setConf(conf)
+    }
     val reader = format.createRecordReader(
       split.serializableHadoopSplit.value, hadoopAttemptContext)
     reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
 
     // Register an on-task-completion callback to close the input stream.
-    context.addOnCompleteCallback(() => reader.close())
+    context.addOnCompleteCallback(() => close())
 
     var havePair = false
     var finished = false
@@ -84,12 +88,18 @@ class NewHadoopRDD[K, V](
       havePair = false
       return (reader.getCurrentKey, reader.getCurrentValue)
     }
+
+    private def close() {
+      try {
+        reader.close()
+      } catch {
+        case e: Exception => logWarning("Exception in RecordReader.close()", e)
+      }
+    }
   }
 
-  override def preferredLocations(split: Split) = {
-    val theSplit = split.asInstanceOf[NewHadoopSplit]
+  override def getPreferredLocations(split: Partition): Seq[String] = {
+    val theSplit = split.asInstanceOf[NewHadoopPartition]
     theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost")
   }
-
-  override val dependencies: List[Dependency[_]] = Nil
 }
diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
similarity index 69%
rename from core/src/main/scala/spark/ParallelCollection.scala
rename to core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
index a27f766e31bd98dacaf137da052e1f02ee275c68..07585a88ceb36930ee626e505d53b5030bb8b02f 100644
--- a/core/src/main/scala/spark/ParallelCollection.scala
+++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
@@ -1,52 +1,53 @@
-package spark
+package spark.rdd
 
 import scala.collection.immutable.NumericRange
 import scala.collection.mutable.ArrayBuffer
+import scala.collection.Map
+import spark.{RDD, TaskContext, SparkContext, Partition}
 
-private[spark] class ParallelCollectionSplit[T: ClassManifest](
+private[spark] class ParallelCollectionPartition[T: ClassManifest](
     val rddId: Long,
     val slice: Int,
     values: Seq[T])
-  extends Split with Serializable {
+  extends Partition with Serializable {
 
   def iterator: Iterator[T] = values.iterator
 
   override def hashCode(): Int = (41 * (41 + rddId) + slice).toInt
 
   override def equals(other: Any): Boolean = other match {
-    case that: ParallelCollectionSplit[_] => (this.rddId == that.rddId && this.slice == that.slice)
+    case that: ParallelCollectionPartition[_] => (this.rddId == that.rddId && this.slice == that.slice)
     case _ => false
   }
 
   override val index: Int = slice
 }
 
-private[spark] class ParallelCollection[T: ClassManifest](
-    sc: SparkContext,
+private[spark] class ParallelCollectionRDD[T: ClassManifest](
+    @transient sc: SparkContext,
     @transient data: Seq[T],
-    numSlices: Int)
-  extends RDD[T](sc) {
+    numSlices: Int,
+    locationPrefs: Map[Int,Seq[String]])
+  extends RDD[T](sc, Nil) {
   // TODO: Right now, each split sends along its full data, even if later down the RDD chain it gets
   // cached. It might be worthwhile to write the data to a file in the DFS and read it in the split
   // instead.
+  // UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal.
 
-  @transient
-  val splits_ = {
-    val slices = ParallelCollection.slice(data, numSlices).toArray
-    slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray
+  override def getPartitions: Array[Partition] = {
+    val slices = ParallelCollectionRDD.slice(data, numSlices).toArray
+    slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray
   }
 
-  override def splits = splits_.asInstanceOf[Array[Split]]
+  override def compute(s: Partition, context: TaskContext) =
+    s.asInstanceOf[ParallelCollectionPartition[T]].iterator
 
-  override def compute(s: Split, taskContext: TaskContext) =
-    s.asInstanceOf[ParallelCollectionSplit[T]].iterator
-
-  override def preferredLocations(s: Split): Seq[String] = Nil
-
-  override val dependencies: List[Dependency[_]] = Nil
+  override def getPreferredLocations(s: Partition): Seq[String] = {
+    locationPrefs.getOrElse(s.index, Nil)
+  }
 }
 
-private object ParallelCollection {
+private object ParallelCollectionRDD {
   /**
    * Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range
    * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes
diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..41ff62dd2285749b0829b29a8c9293aee4dbbcfd
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
@@ -0,0 +1,54 @@
+package spark.rdd
+
+import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext}
+
+
+class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition {
+  override val index = idx
+}
+
+
+/**
+ * Represents a dependency between the PartitionPruningRDD and its parent. In this
+ * case, the child RDD contains a subset of partitions of the parents'.
+ */
+class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean)
+  extends NarrowDependency[T](rdd) {
+
+  @transient
+  val partitions: Array[Partition] = rdd.partitions.filter(s => partitionFilterFunc(s.index))
+    .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition }
+
+  override def getParents(partitionId: Int) = List(partitions(partitionId).index)
+}
+
+
+/**
+ * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on
+ * all partitions. An example use case: If we know the RDD is partitioned by range,
+ * and the execution DAG has a filter on the key, we can avoid launching tasks
+ * on partitions that don't have the range covering the key.
+ */
+class PartitionPruningRDD[T: ClassManifest](
+    @transient prev: RDD[T],
+    @transient partitionFilterFunc: Int => Boolean)
+  extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) {
+
+  override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator(
+    split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context)
+
+  override protected def getPartitions: Array[Partition] =
+    getDependencies.head.asInstanceOf[PruneDependency[T]].partitions
+}
+
+
+object PartitionPruningRDD {
+
+  /**
+   * Create a PartitionPruningRDD. This function can be used to create the PartitionPruningRDD
+   * when its type T is not known at compile time.
+   */
+  def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = {
+    new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassManifest)
+  }
+}
diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala
index 336e193217e8186f717ab63fba6bb28ff25523f4..962a1b21ad1d3ff8c32e461984ec444330e4bf67 100644
--- a/core/src/main/scala/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/spark/rdd/PipedRDD.scala
@@ -8,7 +8,7 @@ import scala.collection.JavaConversions._
 import scala.collection.mutable.ArrayBuffer
 import scala.io.Source
 
-import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext}
+import spark.{RDD, SparkEnv, Partition, TaskContext}
 
 
 /**
@@ -16,20 +16,20 @@ import spark.{OneToOneDependency, RDD, SparkEnv, Split, TaskContext}
  * (printing them one per line) and returns the output as a collection of strings.
  */
 class PipedRDD[T: ClassManifest](
-  parent: RDD[T], command: Seq[String], envVars: Map[String, String])
-  extends RDD[String](parent.context) {
+    prev: RDD[T],
+    command: Seq[String],
+    envVars: Map[String, String])
+  extends RDD[String](prev) {
 
-  def this(parent: RDD[T], command: Seq[String]) = this(parent, command, Map())
+  def this(prev: RDD[T], command: Seq[String]) = this(prev, command, Map())
 
   // Similar to Runtime.exec(), if we are given a single string, split it into words
   // using a standard StringTokenizer (i.e. by spaces)
-  def this(parent: RDD[T], command: String) = this(parent, PipedRDD.tokenize(command))
+  def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command))
 
-  override def splits = parent.splits
+  override def getPartitions: Array[Partition] = firstParent[T].partitions
 
-  override val dependencies = List(new OneToOneDependency(parent))
-
-  override def compute(split: Split, context: TaskContext): Iterator[String] = {
+  override def compute(split: Partition, context: TaskContext): Iterator[String] = {
     val pb = new ProcessBuilder(command)
     // Add the environmental variables to the process.
     val currentEnvVars = pb.environment()
@@ -52,7 +52,7 @@ class PipedRDD[T: ClassManifest](
       override def run() {
         SparkEnv.set(env)
         val out = new PrintWriter(proc.getOutputStream)
-        for (elem <- parent.iterator(split, context)) {
+        for (elem <- firstParent[T].iterator(split, context)) {
           out.println(elem)
         }
         out.close()
diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala
index 6e4797aabbec436c90b2faea9d8078c0e90f3ce6..243673f1518729f6228f8f857ad6fc19b29fd08e 100644
--- a/core/src/main/scala/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/spark/rdd/SampledRDD.scala
@@ -1,44 +1,39 @@
 package spark.rdd
 
 import java.util.Random
+
 import cern.jet.random.Poisson
 import cern.jet.random.engine.DRand
 
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
-
+import spark.{RDD, Partition, TaskContext}
 
 private[spark]
-class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable {
+class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable {
   override val index: Int = prev.index
 }
 
 class SampledRDD[T: ClassManifest](
     prev: RDD[T],
-    withReplacement: Boolean,
+    withReplacement: Boolean, 
     frac: Double,
     seed: Int)
-  extends RDD[T](prev.context) {
+  extends RDD[T](prev) {
 
-  @transient
-  val splits_ = {
+  override def getPartitions: Array[Partition] = {
     val rg = new Random(seed)
-    prev.splits.map(x => new SampledRDDSplit(x, rg.nextInt))
+    firstParent[T].partitions.map(x => new SampledRDDPartition(x, rg.nextInt))
   }
 
-  override def splits = splits_.asInstanceOf[Array[Split]]
-
-  override val dependencies = List(new OneToOneDependency(prev))
-
-  override def preferredLocations(split: Split) =
-    prev.preferredLocations(split.asInstanceOf[SampledRDDSplit].prev)
+  override def getPreferredLocations(split: Partition): Seq[String] =
+    firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDPartition].prev)
 
-  override def compute(splitIn: Split, context: TaskContext) = {
-    val split = splitIn.asInstanceOf[SampledRDDSplit]
+  override def compute(splitIn: Partition, context: TaskContext): Iterator[T] = {
+    val split = splitIn.asInstanceOf[SampledRDDPartition]
     if (withReplacement) {
       // For large datasets, the expected number of occurrences of each element in a sample with
       // replacement is Poisson(frac). We use that to get a count for each element.
       val poisson = new Poisson(frac, new DRand(split.seed))
-      prev.iterator(split.prev, context).flatMap { element =>
+      firstParent[T].iterator(split.prev, context).flatMap { element =>
         val count = poisson.nextInt()
         if (count == 0) {
           Iterator.empty  // Avoid object allocation when we return 0 items, which is quite often
@@ -48,7 +43,7 @@ class SampledRDD[T: ClassManifest](
       }
     } else { // Sampling without replacement
       val rand = new Random(split.seed)
-      prev.iterator(split.prev, context).filter(x => (rand.nextDouble <= frac))
+      firstParent[T].iterator(split.prev, context).filter(x => (rand.nextDouble <= frac))
     }
   }
 }
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
index f83263364684ea2d87489801f6395252de34c46f..4e33b7dd5ca844b4dc425b72fd1f09a2893e5e1d 100644
--- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
@@ -1,37 +1,33 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext}
+import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}
+import spark.SparkContext._
 
-
-private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
+private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
   override val index = idx
   override def hashCode(): Int = idx
 }
 
 /**
  * The resulting RDD from a shuffle (e.g. repartitioning of data).
- * @param parent the parent RDD.
+ * @param prev the parent RDD.
  * @param part the partitioner used to partition the RDD
  * @tparam K the key class.
  * @tparam V the value class.
  */
 class ShuffledRDD[K, V](
-    @transient parent: RDD[(K, V)],
-    part: Partitioner) extends RDD[(K, V)](parent.context) {
+    @transient prev: RDD[(K, V)],
+    part: Partitioner)
+  extends RDD[(K, V)](prev.context, List(new ShuffleDependency(prev, part))) {
 
   override val partitioner = Some(part)
 
-  @transient
-  val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
-
-  override def splits = splits_
-
-  override def preferredLocations(split: Split) = Nil
-
-  val dep = new ShuffleDependency(parent, part)
-  override val dependencies = List(dep)
+  override def getPartitions: Array[Partition] = {
+    Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i))
+  }
 
-  override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = {
-    SparkEnv.get.shuffleFetcher.fetch[K, V](dep.shuffleId, split.index)
+  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)
   }
 }
diff --git a/core/src/main/scala/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
new file mode 100644
index 0000000000000000000000000000000000000000..481e03b349af0d7681ef5d25d6f3ac07bfec69f5
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
@@ -0,0 +1,100 @@
+package spark.rdd
+
+import java.util.{HashMap => JHashMap}
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+import spark.RDD
+import spark.Partitioner
+import spark.Dependency
+import spark.TaskContext
+import spark.Partition
+import spark.SparkEnv
+import spark.ShuffleDependency
+import spark.OneToOneDependency
+
+/**
+ * An optimized version of cogroup for set difference/subtraction.
+ *
+ * It is possible to implement this operation with just `cogroup`, but
+ * that is less efficient because all of the entries from `rdd2`, for
+ * both matching and non-matching values in `rdd1`, are kept in the
+ * JHashMap until the end.
+ *
+ * With this implementation, only the entries from `rdd1` are kept in-memory,
+ * and the entries from `rdd2` are essentially streamed, as we only need to
+ * touch each once to decide if the value needs to be removed.
+ *
+ * This is particularly helpful when `rdd1` is much smaller than `rdd2`, as
+ * 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[K: ClassManifest, V: ClassManifest, W: ClassManifest](
+    @transient var rdd1: RDD[(K, V)],
+    @transient var rdd2: RDD[(K, W)],
+    part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) {
+
+  override def getDependencies: Seq[Dependency[_]] = {
+    Seq(rdd1, rdd2).map { rdd =>
+      if (rdd.partitioner == Some(part)) {
+        logInfo("Adding one-to-one dependency with " + rdd)
+        new OneToOneDependency(rdd)
+      } else {
+        logInfo("Adding shuffle dependency with " + rdd)
+        new ShuffleDependency(rdd.asInstanceOf[RDD[(K, Any)]], part)
+      }
+    }
+  }
+
+  override def getPartitions: Array[Partition] = {
+    val array = new Array[Partition](part.numPartitions)
+    for (i <- 0 until array.size) {
+      // Each CoGroupPartition will depend on rdd1 and rdd2
+      array(i) = new CoGroupPartition(i, Seq(rdd1, rdd2).zipWithIndex.map { case (rdd, j) =>
+        dependencies(j) match {
+          case s: ShuffleDependency[_, _] =>
+            new ShuffleCoGroupSplitDep(s.shuffleId)
+          case _ =>
+            new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))
+        }
+      }.toArray)
+    }
+    array
+  }
+
+  override val partitioner = Some(part)
+
+  override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = {
+    val partition = p.asInstanceOf[CoGroupPartition]
+    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) =>
+        for (t <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index, context.taskMetrics))
+          op(t.asInstanceOf[(K, V)])
+    }
+    // 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() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+  }
+
+}
diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala
index a08473f7beb098f8cac3104d1cc07e8e3191bb76..2c52a67e22635bf196a2d5341bf0972a2ab3075c 100644
--- a/core/src/main/scala/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/spark/rdd/UnionRDD.scala
@@ -1,55 +1,56 @@
 package spark.rdd
 
 import scala.collection.mutable.ArrayBuffer
+import spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
 
-import spark.{Dependency, RangeDependency, RDD, SparkContext, Split, TaskContext}
+private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
+  extends Partition {
 
-
-private[spark] class UnionSplit[T: ClassManifest](
-    idx: Int,
-    rdd: RDD[T],
-    split: Split)
-  extends Split
-  with Serializable {
+  var split: Partition = rdd.partitions(splitIndex)
 
   def iterator(context: TaskContext) = rdd.iterator(split, context)
+
   def preferredLocations() = rdd.preferredLocations(split)
+
   override val index: Int = idx
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent split at the time of task serialization
+    split = rdd.partitions(splitIndex)
+    oos.defaultWriteObject()
+  }
 }
 
 class UnionRDD[T: ClassManifest](
     sc: SparkContext,
-    @transient rdds: Seq[RDD[T]])
-  extends RDD[T](sc)
-  with Serializable {
+    @transient var rdds: Seq[RDD[T]])
+  extends RDD[T](sc, Nil) {  // Nil since we implement getDependencies
 
-  @transient
-  val splits_ : Array[Split] = {
-    val array = new Array[Split](rdds.map(_.splits.size).sum)
+  override def getPartitions: Array[Partition] = {
+    val array = new Array[Partition](rdds.map(_.partitions.size).sum)
     var pos = 0
-    for (rdd <- rdds; split <- rdd.splits) {
-      array(pos) = new UnionSplit(pos, rdd, split)
+    for (rdd <- rdds; split <- rdd.partitions) {
+      array(pos) = new UnionPartition(pos, rdd, split.index)
       pos += 1
     }
     array
   }
 
-  override def splits = splits_
-
-  @transient
-  override val dependencies = {
+  override def getDependencies: Seq[Dependency[_]] = {
     val deps = new ArrayBuffer[Dependency[_]]
     var pos = 0
     for (rdd <- rdds) {
-      deps += new RangeDependency(rdd, 0, pos, rdd.splits.size)
-      pos += rdd.splits.size
+      deps += new RangeDependency(rdd, 0, pos, rdd.partitions.size)
+      pos += rdd.partitions.size
     }
-    deps.toList
+    deps
   }
 
-  override def compute(s: Split, context: TaskContext): Iterator[T] =
-    s.asInstanceOf[UnionSplit[T]].iterator(context)
+  override def compute(s: Partition, context: TaskContext): Iterator[T] =
+    s.asInstanceOf[UnionPartition[T]].iterator(context)
 
-  override def preferredLocations(s: Split): Seq[String] =
-    s.asInstanceOf[UnionSplit[T]].preferredLocations()
+  override def getPreferredLocations(s: Partition): Seq[String] =
+    s.asInstanceOf[UnionPartition[T]].preferredLocations()
 }
diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala
index 92d667ff1e389e2697b78af7bd981e1936badca6..35b0e06785cf370b04a718b4874cf2302bdf99c3 100644
--- a/core/src/main/scala/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala
@@ -1,53 +1,60 @@
 package spark.rdd
 
-import spark.{OneToOneDependency, RDD, SparkContext, Split, TaskContext}
+import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
+import java.io.{ObjectOutputStream, IOException}
 
 
-private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest](
+private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
     idx: Int,
-    rdd1: RDD[T],
-    rdd2: RDD[U],
-    split1: Split,
-    split2: Split)
-  extends Split
-  with Serializable {
+    @transient rdd1: RDD[T],
+    @transient rdd2: RDD[U]
+  ) extends Partition {
 
-  def iterator(context: TaskContext): Iterator[(T, U)] =
-    rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
+  var partition1 = rdd1.partitions(idx)
+  var partition2 = rdd2.partitions(idx)
+  override val index: Int = idx
 
-  def preferredLocations(): Seq[String] =
-    rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
+  def partitions = (partition1, partition2)
 
-  override val index: Int = idx
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    // Update the reference to parent partition at the time of task serialization
+    partition1 = rdd1.partitions(idx)
+    partition2 = rdd2.partitions(idx)
+    oos.defaultWriteObject()
+  }
 }
 
 class ZippedRDD[T: ClassManifest, U: ClassManifest](
     sc: SparkContext,
-    @transient rdd1: RDD[T],
-    @transient rdd2: RDD[U])
-  extends RDD[(T, U)](sc)
-  with Serializable {
+    var rdd1: RDD[T],
+    var rdd2: RDD[U])
+  extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) {
 
-  @transient
-  val splits_ : Array[Split] = {
-    if (rdd1.splits.size != rdd2.splits.size) {
+  override def getPartitions: Array[Partition] = {
+    if (rdd1.partitions.size != rdd2.partitions.size) {
       throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions")
     }
-    val array = new Array[Split](rdd1.splits.size)
-    for (i <- 0 until rdd1.splits.size) {
-      array(i) = new ZippedSplit(i, rdd1, rdd2, rdd1.splits(i), rdd2.splits(i))
+    val array = new Array[Partition](rdd1.partitions.size)
+    for (i <- 0 until rdd1.partitions.size) {
+      array(i) = new ZippedPartition(i, rdd1, rdd2)
     }
     array
   }
 
-  override def splits = splits_
-
-  @transient
-  override val dependencies = List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))
+  override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = {
+    val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions
+    rdd1.iterator(partition1, context).zip(rdd2.iterator(partition2, context))
+  }
 
-  override def compute(s: Split, context: TaskContext): Iterator[(T, U)] =
-    s.asInstanceOf[ZippedSplit[T, U]].iterator(context)
+  override def getPreferredLocations(s: Partition): Seq[String] = {
+    val (partition1, partition2) = s.asInstanceOf[ZippedPartition[T, U]].partitions
+    rdd1.preferredLocations(partition1).intersect(rdd2.preferredLocations(partition2))
+  }
 
-  override def preferredLocations(s: Split): Seq[String] =
-    s.asInstanceOf[ZippedSplit[T, U]].preferredLocations()
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+  }
 }
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index 29757b1178591d5dd79f8ff6d62880ecfee7b8d0..c54dce51d783969e09f6b924db09aefc1352e6a4 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -1,19 +1,19 @@
 package spark.scheduler
 
-import java.net.URI
+import cluster.TaskInfo
 import java.util.concurrent.atomic.AtomicInteger
-import java.util.concurrent.Future
 import java.util.concurrent.LinkedBlockingQueue
 import java.util.concurrent.TimeUnit
 
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue, Map}
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
 
 import spark._
+import spark.executor.TaskMetrics
 import spark.partial.ApproximateActionListener
 import spark.partial.ApproximateEvaluator
 import spark.partial.PartialResult
 import spark.storage.BlockManagerMaster
-import spark.storage.BlockManagerId
+import spark.util.{MetadataCleaner, TimeStampedHashMap}
 
 /**
  * A Scheduler subclass that implements stage-oriented scheduling. It computes a DAG of stages for
@@ -22,7 +22,16 @@ import spark.storage.BlockManagerId
  * and to report fetch failures (the submitTasks method, and code to add CompletionEvents).
  */
 private[spark]
-class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with Logging {
+class DAGScheduler(
+    taskSched: TaskScheduler,
+    mapOutputTracker: MapOutputTracker,
+    blockManagerMaster: BlockManagerMaster,
+    env: SparkEnv)
+  extends TaskSchedulerListener with Logging {
+
+  def this(taskSched: TaskScheduler) {
+    this(taskSched, SparkEnv.get.mapOutputTracker, SparkEnv.get.blockManager.master, SparkEnv.get)
+  }
   taskSched.setListener(this)
 
   // Called by TaskScheduler to report task completions or failures.
@@ -30,16 +39,18 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
       task: Task[_],
       reason: TaskEndReason,
       result: Any,
-      accumUpdates: Map[Long, Any]) {
-    eventQueue.put(CompletionEvent(task, reason, result, accumUpdates))
+      accumUpdates: Map[Long, Any],
+      taskInfo: TaskInfo,
+      taskMetrics: TaskMetrics) {
+    eventQueue.put(CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics))
   }
 
-  // Called by TaskScheduler when a host fails.
-  override def hostLost(host: String) {
-    eventQueue.put(HostLost(host))
+  // Called by TaskScheduler when an executor fails.
+  override def executorLost(execId: String) {
+    eventQueue.put(ExecutorLost(execId))
   }
 
-  // Called by TaskScheduler to cancel an entier TaskSet due to repeated failures.
+  // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures.
   override def taskSetFailed(taskSet: TaskSet, reason: String) {
     eventQueue.put(TaskSetFailed(taskSet, reason))
   }
@@ -53,50 +64,63 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
   // resubmit failed stages
   val POLL_TIMEOUT = 10L
 
-  private val lock = new Object          // Used for access to the entire DAGScheduler
-
   private val eventQueue = new LinkedBlockingQueue[DAGSchedulerEvent]
 
   val nextRunId = new AtomicInteger(0)
 
   val nextStageId = new AtomicInteger(0)
 
-  val idToStage = new HashMap[Int, Stage]
+  val idToStage = new TimeStampedHashMap[Int, Stage]
 
-  val shuffleToMapStage = new HashMap[Int, Stage]
+  val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
 
-  var cacheLocs = new HashMap[Int, Array[List[String]]]
+  private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo]
 
-  val env = SparkEnv.get
-  val cacheTracker = env.cacheTracker
-  val mapOutputTracker = env.mapOutputTracker
+  private[spark] val sparkListeners = ArrayBuffer[SparkListener]()
+
+  var cacheLocs = new HashMap[Int, Array[List[String]]]
 
-  val deadHosts = new HashSet[String]  // TODO: The code currently assumes these can't come back;
-                                       // that's not going to be a realistic assumption in general
+  // For tracking failed nodes, we use the MapOutputTracker's generation number, which is
+  // sent with every task. When we detect a node failing, we note the current generation number
+  // and failed executor, increment it for new tasks, and use this to ignore stray ShuffleMapTask
+  // results.
+  // TODO: Garbage collect information about failure generations when we know there are no more
+  //       stray messages to detect.
+  val failedGeneration = new HashMap[String, Long]
 
   val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
   val running = new HashSet[Stage] // Stages we are running right now
   val failed = new HashSet[Stage]  // Stages that must be resubmitted due to fetch failures
-  val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
+  val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
   var lastFetchFailureTime: Long = 0  // Used to wait a bit to avoid repeated resubmits
 
   val activeJobs = new HashSet[ActiveJob]
   val resultStageToJob = new HashMap[Stage, ActiveJob]
 
+  val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup)
+
   // Start a thread to run the DAGScheduler event loop
-  new Thread("DAGScheduler") {
-    setDaemon(true)
-    override def run() {
-      DAGScheduler.this.run()
-    }
-  }.start()
+  def start() {
+    new Thread("DAGScheduler") {
+      setDaemon(true)
+      override def run() {
+        DAGScheduler.this.run()
+      }
+    }.start()
+  }
 
-  def getCacheLocs(rdd: RDD[_]): Array[List[String]] = {
+  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
+    }
     cacheLocs(rdd.id)
   }
 
-  def updateCacheLocs() {
-    cacheLocs = cacheTracker.getLocationsSnapshot()
+  private def clearCacheLocs() {
+    cacheLocs.clear()
   }
 
   /**
@@ -104,7 +128,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
    * The priority value passed in will be used if the stage doesn't already exist with
    * a lower priority (we assume that priorities always increase across jobs for now).
    */
-  def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = {
+  private def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = {
     shuffleToMapStage.get(shuffleDep.shuffleId) match {
       case Some(stage) => stage
       case None =>
@@ -119,17 +143,17 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
    * as a result stage for the final RDD used directly in an action. The stage will also be given
    * the provided priority.
    */
-  def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = {
-    // Kind of ugly: need to register RDDs with the cache and map output tracker here
-    // since we can't do it in the RDD constructor because # of splits is unknown
-    logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
-    cacheTracker.registerRDD(rdd.id, rdd.splits.size)
+  private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = {
     if (shuffleDep != None) {
-      mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size)
+      // Kind of ugly: need to register RDDs with the cache and map output tracker here
+      // since we can't do it in the RDD constructor because # of partitions is unknown
+      logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
+      mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
     }
     val id = nextStageId.getAndIncrement()
     val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority)
     idToStage(id) = stage
+    stageToInfos(stage) = StageInfo(stage)
     stage
   }
 
@@ -137,16 +161,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
    * Get or create the list of parent stages for a given RDD. The stages will be assigned the
    * provided priority if they haven't already been created with a lower priority.
    */
-  def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = {
+  private def getParentStages(rdd: RDD[_], priority: Int): List[Stage] = {
     val parents = new HashSet[Stage]
     val visited = new HashSet[RDD[_]]
     def visit(r: RDD[_]) {
       if (!visited(r)) {
         visited += r
         // Kind of ugly: need to register RDDs with the cache here since
-        // we can't do it in its constructor because # of splits is unknown
-        logInfo("Registering parent RDD " + r.id + " (" + r.origin + ")")
-        cacheTracker.registerRDD(r.id, r.splits.size)
+        // we can't do it in its constructor because # of partitions is unknown
         for (dep <- r.dependencies) {
           dep match {
             case shufDep: ShuffleDependency[_,_] =>
@@ -161,25 +183,22 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
     parents.toList
   }
 
-  def getMissingParentStages(stage: Stage): List[Stage] = {
+  private def getMissingParentStages(stage: Stage): List[Stage] = {
     val missing = new HashSet[Stage]
     val visited = new HashSet[RDD[_]]
     def visit(rdd: RDD[_]) {
       if (!visited(rdd)) {
         visited += rdd
-        val locs = getCacheLocs(rdd)
-        for (p <- 0 until rdd.splits.size) {
-          if (locs(p) == Nil) {
-            for (dep <- rdd.dependencies) {
-              dep match {
-                case shufDep: ShuffleDependency[_,_] =>
-                  val mapStage = getShuffleMapStage(shufDep, stage.priority)
-                  if (!mapStage.isAvailable) {
-                    missing += mapStage
-                  }
-                case narrowDep: NarrowDependency[_] =>
-                  visit(narrowDep.rdd)
-              }
+        if (getCacheLocs(rdd).contains(Nil)) {
+          for (dep <- rdd.dependencies) {
+            dep match {
+              case shufDep: ShuffleDependency[_,_] =>
+                val mapStage = getShuffleMapStage(shufDep, stage.priority)
+                if (!mapStage.isAvailable) {
+                  missing += mapStage
+                }
+              case narrowDep: NarrowDependency[_] =>
+                visit(narrowDep.rdd)
             }
           }
         }
@@ -189,23 +208,45 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
     missing.toList
   }
 
+  /**
+   * Returns (and does not submit) a JobSubmitted event suitable to run a given job, and a
+   * JobWaiter whose getResult() method will return the result of the job when it is complete.
+   *
+   * The job is assumed to have at least one partition; zero partition jobs should be handled
+   * without a JobSubmitted event.
+   */
+  private[scheduler] def prepareJob[T, U: ClassManifest](
+      finalRdd: RDD[T],
+      func: (TaskContext, Iterator[T]) => U,
+      partitions: Seq[Int],
+      callSite: String,
+      allowLocal: Boolean,
+      resultHandler: (Int, U) => Unit)
+    : (JobSubmitted, JobWaiter[U]) =
+  {
+    assert(partitions.size > 0)
+    val waiter = new JobWaiter(partitions.size, resultHandler)
+    val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
+    val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter)
+    return (toSubmit, waiter)
+  }
+
   def runJob[T, U: ClassManifest](
       finalRdd: RDD[T],
       func: (TaskContext, Iterator[T]) => U,
       partitions: Seq[Int],
       callSite: String,
-      allowLocal: Boolean)
-    : Array[U] =
+      allowLocal: Boolean,
+      resultHandler: (Int, U) => Unit)
   {
     if (partitions.size == 0) {
-      return new Array[U](0)
+      return
     }
-    val waiter = new JobWaiter(partitions.size)
-    val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
-    eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter))
-    waiter.getResult() match {
-      case JobSucceeded(results: Seq[_]) =>
-        return results.asInstanceOf[Seq[U]].toArray
+    val (toSubmit, waiter) = prepareJob(
+        finalRdd, func, partitions, callSite, allowLocal, resultHandler)
+    eventQueue.put(toSubmit)
+    waiter.awaitResult() match {
+      case JobSucceeded => {}
       case JobFailed(exception: Exception) =>
         logInfo("Failed to run " + callSite)
         throw exception
@@ -222,92 +263,119 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
   {
     val listener = new ApproximateActionListener(rdd, func, evaluator, timeout)
     val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
-    val partitions = (0 until rdd.splits.size).toArray
+    val partitions = (0 until rdd.partitions.size).toArray
     eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener))
-    return listener.getResult()    // Will throw an exception if the job fails
+    return listener.awaitResult()    // Will throw an exception if the job fails
+  }
+
+  /**
+   * Process one event retrieved from the event queue.
+   * Returns true if we should stop the event loop.
+   */
+  private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = {
+    event match {
+      case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) =>
+        val runId = nextRunId.getAndIncrement()
+        val finalStage = newStage(finalRDD, None, runId)
+        val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener)
+        clearCacheLocs()
+        logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length +
+                " output partitions (allowLocal=" + allowLocal + ")")
+        logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")")
+        logInfo("Parents of final stage: " + finalStage.parents)
+        logInfo("Missing parents: " + getMissingParentStages(finalStage))
+        if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) {
+          // Compute very short actions like first() or take() with no parent stages locally.
+          runLocally(job)
+        } else {
+          activeJobs += job
+          resultStageToJob(finalStage) = job
+          submitStage(finalStage)
+        }
+
+      case ExecutorLost(execId) =>
+        handleExecutorLost(execId)
+
+      case completion: CompletionEvent =>
+        handleTaskCompletion(completion)
+
+      case TaskSetFailed(taskSet, reason) =>
+        abortStage(idToStage(taskSet.stageId), reason)
+
+      case StopDAGScheduler =>
+        // Cancel any active jobs
+        for (job <- activeJobs) {
+          val error = new SparkException("Job cancelled because SparkContext was shut down")
+          job.listener.jobFailed(error)
+        }
+        return true
+    }
+    return false
+  }
+
+  /**
+   * Resubmit any failed stages. Ordinarily called after a small amount of time has passed since
+   * the last fetch failure.
+   */
+  private[scheduler] def resubmitFailedStages() {
+    logInfo("Resubmitting failed stages")
+    clearCacheLocs()
+    val failed2 = failed.toArray
+    failed.clear()
+    for (stage <- failed2.sortBy(_.priority)) {
+      submitStage(stage)
+    }
+  }
+  
+  /**
+   * Check for waiting or failed stages which are now eligible for resubmission.
+   * Ordinarily run on every iteration of the event loop.
+   */
+  private[scheduler] def submitWaitingStages() {
+    // TODO: We might want to run this less often, when we are sure that something has become
+    // runnable that wasn't before.
+    logTrace("Checking for newly runnable parent stages")
+    logTrace("running: " + running)
+    logTrace("waiting: " + waiting)
+    logTrace("failed: " + failed)
+    val waiting2 = waiting.toArray
+    waiting.clear()
+    for (stage <- waiting2.sortBy(_.priority)) {
+      submitStage(stage)
+    }
   }
 
+
   /**
    * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure
    * events and responds by launching tasks. This runs in a dedicated thread and receives events
    * via the eventQueue.
    */
-  def run() {
+  private def run() {
     SparkEnv.set(env)
 
     while (true) {
       val event = eventQueue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)
-      val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability
       if (event != null) {
         logDebug("Got event of type " + event.getClass.getName)
       }
 
-      event match {
-        case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) =>
-          val runId = nextRunId.getAndIncrement()
-          val finalStage = newStage(finalRDD, None, runId)
-          val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener)
-          updateCacheLocs()
-          logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length +
-                  " output partitions")
-          logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")")
-          logInfo("Parents of final stage: " + finalStage.parents)
-          logInfo("Missing parents: " + getMissingParentStages(finalStage))
-          if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) {
-            // Compute very short actions like first() or take() with no parent stages locally.
-            runLocally(job)
-          } else {
-            activeJobs += job
-            resultStageToJob(finalStage) = job
-            submitStage(finalStage)
-          }
-
-        case HostLost(host) =>
-          handleHostLost(host)
-
-        case completion: CompletionEvent =>
-          handleTaskCompletion(completion)
-
-        case TaskSetFailed(taskSet, reason) =>
-          abortStage(idToStage(taskSet.stageId), reason)
-
-        case StopDAGScheduler =>
-          // Cancel any active jobs
-          for (job <- activeJobs) {
-            val error = new SparkException("Job cancelled because SparkContext was shut down")
-            job.listener.jobFailed(error)
-          }
+      if (event != null) {
+        if (processEvent(event)) {
           return
-
-        case null =>
-          // queue.poll() timed out, ignore it
+        }
       }
 
+      val time = System.currentTimeMillis() // TODO: use a pluggable clock for testability
       // Periodically resubmit failed stages if some map output fetches have failed and we have
       // waited at least RESUBMIT_TIMEOUT. We wait for this short time because when a node fails,
       // tasks on many other nodes are bound to get a fetch failure, and they won't all get it at
       // the same time, so we want to make sure we've identified all the reduce tasks that depend
       // on the failed node.
       if (failed.size > 0 && time > lastFetchFailureTime + RESUBMIT_TIMEOUT) {
-        logInfo("Resubmitting failed stages")
-        updateCacheLocs()
-        val failed2 = failed.toArray
-        failed.clear()
-        for (stage <- failed2.sortBy(_.priority)) {
-          submitStage(stage)
-        }
+        resubmitFailedStages()
       } else {
-        // TODO: We might want to run this less often, when we are sure that something has become
-        // runnable that wasn't before.
-        logDebug("Checking for newly runnable parent stages")
-        logDebug("running: " + running)
-        logDebug("waiting: " + waiting)
-        logDebug("failed: " + failed)
-        val waiting2 = waiting.toArray
-        waiting.clear()
-        for (stage <- waiting2.sortBy(_.priority)) {
-          submitStage(stage)
-        }
+        submitWaitingStages()
       }
     }
   }
@@ -317,33 +385,42 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
    * We run the operation in a separate thread just in case it takes a bunch of time, so that we
    * don't block the DAGScheduler event loop or other concurrent jobs.
    */
-  def runLocally(job: ActiveJob) {
+  protected def runLocally(job: ActiveJob) {
     logInfo("Computing the requested partition locally")
     new Thread("Local computation of job " + job.runId) {
       override def run() {
-        try {
-          SparkEnv.set(env)
-          val rdd = job.finalStage.rdd
-          val split = rdd.splits(job.partitions(0))
-          val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0)
-          val result = job.func(taskContext, rdd.iterator(split, taskContext))
-          taskContext.executeOnCompleteCallbacks()
-          job.listener.taskSucceeded(0, result)
-        } catch {
-          case e: Exception =>
-            job.listener.jobFailed(e)
-        }
+        runLocallyWithinThread(job)
       }
     }.start()
   }
 
-  def submitStage(stage: Stage) {
+  // Broken out for easier testing in DAGSchedulerSuite.
+  protected def runLocallyWithinThread(job: ActiveJob) {
+    try {
+      SparkEnv.set(env)
+      val rdd = job.finalStage.rdd
+      val split = rdd.partitions(job.partitions(0))
+      val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0)
+      try {
+        val result = job.func(taskContext, rdd.iterator(split, taskContext))
+        job.listener.taskSucceeded(0, result)
+      } finally {
+        taskContext.executeOnCompleteCallbacks()
+      }
+    } catch {
+      case e: Exception =>
+        job.listener.jobFailed(e)
+    }
+  }
+
+  /** Submits stage, but first recursively submits any missing parents. */
+  private def submitStage(stage: Stage) {
     logDebug("submitStage(" + stage + ")")
     if (!waiting(stage) && !running(stage) && !failed(stage)) {
       val missing = getMissingParentStages(stage).sortBy(_.id)
       logDebug("missing: " + missing)
       if (missing == Nil) {
-        logInfo("Submitting " + stage + " (" + stage.origin + "), which has no missing parents")
+        logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents")
         submitMissingTasks(stage)
         running += stage
       } else {
@@ -355,7 +432,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
     }
   }
 
-  def submitMissingTasks(stage: Stage) {
+  /** Called when stage's parents are available and we can now do its task. */
+  private def submitMissingTasks(stage: Stage) {
     logDebug("submitMissingTasks(" + stage + ")")
     // Get our pending tasks and remember them in our pendingTasks entry
     val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet)
@@ -376,11 +454,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
       }
     }
     if (tasks.size > 0) {
-      logInfo("Submitting " + tasks.size + " missing tasks from " + stage)
+      logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")")
       myPending ++= tasks
       logDebug("New pending tasks: " + myPending)
       taskSched.submitTasks(
         new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.priority))
+      if (!stage.submissionTime.isDefined) {
+        stage.submissionTime = Some(System.currentTimeMillis())
+      }
     } else {
       logDebug("Stage " + stage + " is actually done; %b %d %d".format(
         stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions))
@@ -392,9 +473,20 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
    * Responds to a task finishing. This is called inside the event loop so it assumes that it can
    * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside.
    */
-  def handleTaskCompletion(event: CompletionEvent) {
+  private def handleTaskCompletion(event: CompletionEvent) {
     val task = event.task
     val stage = idToStage(task.stageId)
+
+    def markStageAsFinished(stage: Stage) = {
+      val serviceTime = stage.submissionTime match {
+        case Some(t) => "%.03f".format((System.currentTimeMillis() - t) / 1000.0)
+        case _ => "Unkown"
+      }
+      logInfo("%s (%s) finished in %s s".format(stage, stage.origin, serviceTime))
+      val stageComp = StageCompleted(stageToInfos(stage))
+      sparkListeners.foreach{_.onStageCompleted(stageComp)}
+      running -= stage
+    }
     event.reason match {
       case Success =>
         logInfo("Completed " + task)
@@ -402,6 +494,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
           Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted
         }
         pendingTasks(stage) -= task
+        stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics
         task match {
           case rt: ResultTask[_, _] =>
             resultStageToJob.get(stage) match {
@@ -409,38 +502,46 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
                 if (!job.finished(rt.outputId)) {
                   job.finished(rt.outputId) = true
                   job.numFinished += 1
-                  job.listener.taskSucceeded(rt.outputId, event.result)
                   // If the whole job has finished, remove it
                   if (job.numFinished == job.numPartitions) {
                     activeJobs -= job
                     resultStageToJob -= stage
-                    running -= stage
+                    markStageAsFinished(stage)
                   }
+                  job.listener.taskSucceeded(rt.outputId, event.result)
                 }
               case None =>
                 logInfo("Ignoring result from " + rt + " because its job has finished")
             }
 
           case smt: ShuffleMapTask =>
-            val stage = idToStage(smt.stageId)
             val status = event.result.asInstanceOf[MapStatus]
-            val host = status.address.ip
-            logInfo("ShuffleMapTask finished with host " + host)
-            if (!deadHosts.contains(host)) {   // TODO: Make sure hostnames are consistent with Mesos
+            val execId = status.location.executorId
+            logDebug("ShuffleMapTask finished on " + execId)
+            if (failedGeneration.contains(execId) && smt.generation <= failedGeneration(execId)) {
+              logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId)
+            } else {
               stage.addOutputLoc(smt.partition, status)
             }
             if (running.contains(stage) && pendingTasks(stage).isEmpty) {
-              logInfo(stage + " (" + stage.origin + ") finished; looking for newly runnable stages")
-              running -= stage
+              markStageAsFinished(stage)
+              logInfo("looking for newly runnable stages")
               logInfo("running: " + running)
               logInfo("waiting: " + waiting)
               logInfo("failed: " + failed)
               if (stage.shuffleDep != None) {
+                // We supply true to increment the generation number here in case this is a
+                // recomputation of the map outputs. In that case, some nodes may have cached
+                // locations with holes (from when we detected the error) and will need the
+                // generation incremented to refetch them.
+                // TODO: Only increment the generation number if this is not the first time
+                //       we registered these map outputs.
                 mapOutputTracker.registerMapOutputs(
                   stage.shuffleDep.get.shuffleId,
-                  stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray)
+                  stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray,
+                  true)
               }
-              updateCacheLocs()
+              clearCacheLocs()
               if (stage.outputLocs.count(_ == Nil) != 0) {
                 // Some tasks had failed; let's resubmit this stage
                 // TODO: Lower-level scheduler should also deal with this
@@ -459,7 +560,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
                 waiting --= newlyRunnable
                 running ++= newlyRunnable
                 for (stage <- newlyRunnable.sortBy(_.id)) {
-                  logInfo("Submitting " + stage + " (" + stage.origin + "), which is now runnable")
+                  logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable")
                   submitMissingTasks(stage)
                 }
               }
@@ -490,9 +591,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
         // Remember that a fetch failed now; this is used to resubmit the broken
         // stages later, after a small wait (to give other tasks the chance to fail)
         lastFetchFailureTime = System.currentTimeMillis() // TODO: Use pluggable clock
-        // TODO: mark the host as failed only if there were lots of fetch failures on it
+        // TODO: mark the executor as failed only if there were lots of fetch failures on it
         if (bmAddress != null) {
-          handleHostLost(bmAddress.ip)
+          handleExecutorLost(bmAddress.executorId, Some(task.generation))
         }
 
       case other =>
@@ -502,22 +603,31 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
   }
 
   /**
-   * Responds to a host being lost. This is called inside the event loop so it assumes that it can
-   * modify the scheduler's internal state. Use hostLost() to post a host lost event from outside.
+   * Responds to an executor being lost. This is called inside the event loop, so it assumes it can
+   * modify the scheduler's internal state. Use executorLost() to post a loss event from outside.
+   *
+   * Optionally the generation during which the failure was caught can be passed to avoid allowing
+   * stray fetch failures from possibly retriggering the detection of a node as lost.
    */
-  def handleHostLost(host: String) {
-    if (!deadHosts.contains(host)) {
-      logInfo("Host lost: " + host)
-      deadHosts += host
-      env.blockManager.master.notifyADeadHost(host)
+  private def handleExecutorLost(execId: String, maybeGeneration: Option[Long] = None) {
+    val currentGeneration = maybeGeneration.getOrElse(mapOutputTracker.getGeneration)
+    if (!failedGeneration.contains(execId) || failedGeneration(execId) < currentGeneration) {
+      failedGeneration(execId) = currentGeneration
+      logInfo("Executor lost: %s (generation %d)".format(execId, currentGeneration))
+      blockManagerMaster.removeExecutor(execId)
       // TODO: This will be really slow if we keep accumulating shuffle map stages
       for ((shuffleId, stage) <- shuffleToMapStage) {
-        stage.removeOutputsOnHost(host)
+        stage.removeOutputsOnExecutor(execId)
         val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray
         mapOutputTracker.registerMapOutputs(shuffleId, locs, true)
       }
-      cacheTracker.cacheLost(host)
-      updateCacheLocs()
+      if (shuffleToMapStage.isEmpty) {
+        mapOutputTracker.incrementGeneration()
+      }
+      clearCacheLocs()
+    } else {
+      logDebug("Additional executor lost message for " + execId +
+               "(generation " + currentGeneration + ")")
     }
   }
 
@@ -525,7 +635,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
    * Aborts all jobs depending on a particular Stage. This is called in response to a task set
    * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
    */
-  def abortStage(failedStage: Stage, reason: String) {
+  private def abortStage(failedStage: Stage, reason: String) {
     val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq
     for (resultStage <- dependentStages) {
       val job = resultStageToJob(resultStage)
@@ -541,7 +651,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
   /**
    * Return true if one of stage's ancestors is target.
    */
-  def stageDependsOn(stage: Stage, target: Stage): Boolean = {
+  private def stageDependsOn(stage: Stage, target: Stage): Boolean = {
     if (stage == target) {
       return true
     }
@@ -568,14 +678,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
     visitedRdds.contains(target.rdd)
   }
 
-  def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = {
+  private def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = {
     // If the partition is cached, return the cache locations
     val cached = getCacheLocs(rdd)(partition)
     if (cached != Nil) {
       return cached
     }
     // If the RDD has some placement preferences (as is the case for input RDDs), get those
-    val rddPrefs = rdd.preferredLocations(rdd.splits(partition)).toList
+    val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList
     if (rddPrefs != Nil) {
       return rddPrefs
     }
@@ -594,8 +704,23 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
     return Nil
   }
 
+  private def cleanup(cleanupTime: Long) {
+    var sizeBefore = idToStage.size
+    idToStage.clearOldValues(cleanupTime)
+    logInfo("idToStage " + sizeBefore + " --> " + idToStage.size)
+
+    sizeBefore = shuffleToMapStage.size
+    shuffleToMapStage.clearOldValues(cleanupTime)
+    logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size)
+    
+    sizeBefore = pendingTasks.size
+    pendingTasks.clearOldValues(cleanupTime)
+    logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size)
+  }
+
   def stop() {
     eventQueue.put(StopDAGScheduler)
+    metadataCleaner.cancel()
     taskSched.stop()
   }
 }
diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
index 3422a21d9d6a9664d9f236ab30c1213970ac50de..ed0b9bf178a89c80f5709efd09d288c18fc7bbac 100644
--- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
@@ -1,8 +1,10 @@
 package spark.scheduler
 
+import spark.scheduler.cluster.TaskInfo
 import scala.collection.mutable.Map
 
 import spark._
+import spark.executor.TaskMetrics
 
 /**
  * Types of events that can be handled by the DAGScheduler. The DAGScheduler uses an event queue
@@ -25,10 +27,12 @@ private[spark] case class CompletionEvent(
     task: Task[_],
     reason: TaskEndReason,
     result: Any,
-    accumUpdates: Map[Long, Any])
+    accumUpdates: Map[Long, Any],
+    taskInfo: TaskInfo,
+    taskMetrics: TaskMetrics)
   extends DAGSchedulerEvent
 
-private[spark] case class HostLost(host: String) extends DAGSchedulerEvent
+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/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala
index c4a74e526f0035986fb220ba1e5c598978c821bb..654131ee8417b209f735074856da271b04414233 100644
--- a/core/src/main/scala/spark/scheduler/JobResult.scala
+++ b/core/src/main/scala/spark/scheduler/JobResult.scala
@@ -5,5 +5,5 @@ package spark.scheduler
  */
 private[spark] sealed trait JobResult
 
-private[spark] case class JobSucceeded(results: Seq[_]) extends JobResult
+private[spark] case object JobSucceeded extends JobResult
 private[spark] case class JobFailed(exception: Exception) extends JobResult
diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala
index b3d4feebe5e621cde40260c06a9883ea42fa9958..3cc6a86345fd195828b999f7b33b7c9c31e7d111 100644
--- a/core/src/main/scala/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala
@@ -3,10 +3,12 @@ package spark.scheduler
 import scala.collection.mutable.ArrayBuffer
 
 /**
- * An object that waits for a DAGScheduler job to complete.
+ * An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their
+ * results to the given handler function.
  */
-private[spark] class JobWaiter(totalTasks: Int) extends JobListener {
-  private val taskResults = ArrayBuffer.fill[Any](totalTasks)(null)
+private[spark] class JobWaiter[T](totalTasks: Int, resultHandler: (Int, T) => Unit)
+  extends JobListener {
+
   private var finishedTasks = 0
 
   private var jobFinished = false          // Is the job as a whole finished (succeeded or failed)?
@@ -17,11 +19,11 @@ private[spark] class JobWaiter(totalTasks: Int) extends JobListener {
       if (jobFinished) {
         throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter")
       }
-      taskResults(index) = result
+      resultHandler(index, result.asInstanceOf[T])
       finishedTasks += 1
       if (finishedTasks == totalTasks) {
         jobFinished = true
-        jobResult = JobSucceeded(taskResults)
+        jobResult = JobSucceeded
         this.notifyAll()
       }
     }
@@ -38,7 +40,7 @@ private[spark] class JobWaiter(totalTasks: Int) extends JobListener {
     }
   }
 
-  def getResult(): JobResult = synchronized {
+  def awaitResult(): JobResult = synchronized {
     while (!jobFinished) {
       this.wait()
     }
diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala
index 4532d9497f303891287a842ca6437924aa1dc1eb..203abb917be1cbe25f9998d1f65815b529efebdf 100644
--- a/core/src/main/scala/spark/scheduler/MapStatus.scala
+++ b/core/src/main/scala/spark/scheduler/MapStatus.scala
@@ -8,19 +8,19 @@ import java.io.{ObjectOutput, ObjectInput, Externalizable}
  * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks.
  * The map output sizes are compressed using MapOutputTracker.compressSize.
  */
-private[spark] class MapStatus(var address: BlockManagerId, var compressedSizes: Array[Byte])
+private[spark] class MapStatus(var location: BlockManagerId, var compressedSizes: Array[Byte])
   extends Externalizable {
 
   def this() = this(null, null)  // For deserialization only
 
   def writeExternal(out: ObjectOutput) {
-    address.writeExternal(out)
+    location.writeExternal(out)
     out.writeInt(compressedSizes.length)
     out.write(compressedSizes)
   }
 
   def readExternal(in: ObjectInput) {
-    address = new BlockManagerId(in)
+    location = BlockManagerId(in)
     compressedSizes = new Array[Byte](in.readInt())
     in.readFully(compressedSizes)
   }
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala
index 2aad7956b41c4414ea3a15f57fbeb5d1099039f5..beb21a76fe5c8247b3373e05cb09355141d49ed8 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/spark/scheduler/ResultTask.scala
@@ -1,20 +1,78 @@
 package spark.scheduler
 
 import spark._
+import java.io._
+import util.{MetadataCleaner, TimeStampedHashMap}
+import java.util.zip.{GZIPInputStream, GZIPOutputStream}
+
+private[spark] object ResultTask {
+
+  // A simple map between the stage id to the serialized byte array of a task.
+  // Served as a cache for task serialization because serialization can be
+  // expensive on the master node if it needs to launch thousands of tasks.
+  val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
+
+  val metadataCleaner = new MetadataCleaner("ResultTask", serializedInfoCache.clearOldValues)
+
+  def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
+    synchronized {
+      val old = serializedInfoCache.get(stageId).orNull
+      if (old != null) {
+        return old
+      } else {
+        val out = new ByteArrayOutputStream
+        val ser = SparkEnv.get.closureSerializer.newInstance
+        val objOut = ser.serializeStream(new GZIPOutputStream(out))
+        objOut.writeObject(rdd)
+        objOut.writeObject(func)
+        objOut.close()
+        val bytes = out.toByteArray
+        serializedInfoCache.put(stageId, bytes)
+        return bytes
+      }
+    }
+  }
+
+  def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = {
+    synchronized {
+      val loader = Thread.currentThread.getContextClassLoader
+      val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
+      val ser = SparkEnv.get.closureSerializer.newInstance
+      val objIn = ser.deserializeStream(in)
+      val rdd = objIn.readObject().asInstanceOf[RDD[_]]
+      val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _]
+      return (rdd, func)
+    }
+  }
+
+  def clearCache() {
+    synchronized {
+      serializedInfoCache.clear()
+    }
+  }
+}
+
 
 private[spark] class ResultTask[T, U](
     stageId: Int,
-    rdd: RDD[T],
-    func: (TaskContext, Iterator[T]) => U,
-    val partition: Int,
+    var rdd: RDD[T],
+    var func: (TaskContext, Iterator[T]) => U,
+    var partition: Int,
     @transient locs: Seq[String],
     val outputId: Int)
-  extends Task[U](stageId) {
+  extends Task[U](stageId) with Externalizable {
 
-  val split = rdd.splits(partition)
+  def this() = this(0, null, null, 0, null, 0)
+
+  var split = if (rdd == null) {
+    null
+  } else {
+    rdd.partitions(partition)
+  }
 
   override def run(attemptId: Long): U = {
     val context = new TaskContext(stageId, partition, attemptId)
+    metrics = Some(context.taskMetrics)
     try {
       func(context, rdd.iterator(split, context))
     } finally {
@@ -25,4 +83,31 @@ private[spark] class ResultTask[T, U](
   override def preferredLocations: Seq[String] = locs
 
   override def toString = "ResultTask(" + stageId + ", " + partition + ")"
+
+  override def writeExternal(out: ObjectOutput) {
+    RDDCheckpointData.synchronized {
+      split = rdd.partitions(partition)
+      out.writeInt(stageId)
+      val bytes = ResultTask.serializeInfo(
+        stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _])
+      out.writeInt(bytes.length)
+      out.write(bytes)
+      out.writeInt(partition)
+      out.writeInt(outputId)
+      out.writeObject(split)
+    }
+  }
+
+  override def readExternal(in: ObjectInput) {
+    val stageId = in.readInt()
+    val numBytes = in.readInt()
+    val bytes = new Array[Byte](numBytes)
+    in.readFully(bytes)
+    val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes)
+    rdd = rdd_.asInstanceOf[RDD[T]]
+    func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
+    partition = in.readInt()
+    val outputId = in.readInt()
+    split = in.readObject().asInstanceOf[Partition]
+  }
 }
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
index bd1911fce2e827985b72cf23c59b13b0d33a7147..36d087a4d009c8e6a4fdf85bfc5d6e6e6e3926bf 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
@@ -13,23 +13,27 @@ import com.ning.compress.lzf.LZFInputStream
 import com.ning.compress.lzf.LZFOutputStream
 
 import spark._
+import executor.ShuffleWriteMetrics
 import spark.storage._
+import util.{TimeStampedHashMap, MetadataCleaner}
 
 private[spark] object ShuffleMapTask {
 
   // A simple map between the stage id to the serialized byte array of a task.
   // Served as a cache for task serialization because serialization can be
   // expensive on the master node if it needs to launch thousands of tasks.
-  val serializedInfoCache = new JHashMap[Int, Array[Byte]]
+  val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
+
+  val metadataCleaner = new MetadataCleaner("ShuffleMapTask", serializedInfoCache.clearOldValues)
 
   def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
     synchronized {
-      val old = serializedInfoCache.get(stageId)
+      val old = serializedInfoCache.get(stageId).orNull
       if (old != null) {
         return old
       } else {
         val out = new ByteArrayOutputStream
-        val ser = SparkEnv.get.closureSerializer.newInstance
+        val ser = SparkEnv.get.closureSerializer.newInstance()
         val objOut = ser.serializeStream(new GZIPOutputStream(out))
         objOut.writeObject(rdd)
         objOut.writeObject(dep)
@@ -45,7 +49,7 @@ private[spark] object ShuffleMapTask {
     synchronized {
       val loader = Thread.currentThread.getContextClassLoader
       val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
-      val ser = SparkEnv.get.closureSerializer.newInstance
+      val ser = SparkEnv.get.closureSerializer.newInstance()
       val objIn = ser.deserializeStream(in)
       val rdd = objIn.readObject().asInstanceOf[RDD[_]]
       val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]]
@@ -78,22 +82,25 @@ private[spark] class ShuffleMapTask(
   with Externalizable
   with Logging {
 
-  def this() = this(0, null, null, 0, null)
+  protected def this() = this(0, null, null, 0, null)
 
   var split = if (rdd == null) {
     null
   } else {
-    rdd.splits(partition)
+    rdd.partitions(partition)
   }
 
   override def writeExternal(out: ObjectOutput) {
-    out.writeInt(stageId)
-    val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
-    out.writeInt(bytes.length)
-    out.write(bytes)
-    out.writeInt(partition)
-    out.writeLong(generation)
-    out.writeObject(split)
+    RDDCheckpointData.synchronized {
+      split = rdd.partitions(partition)
+      out.writeInt(stageId)
+      val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
+      out.writeInt(bytes.length)
+      out.write(bytes)
+      out.writeInt(partition)
+      out.writeLong(generation)
+      out.writeObject(split)
+    }
   }
 
   override def readExternal(in: ObjectInput) {
@@ -106,39 +113,45 @@ private[spark] class ShuffleMapTask(
     dep = dep_
     partition = in.readInt()
     generation = in.readLong()
-    split = in.readObject().asInstanceOf[Split]
+    split = in.readObject().asInstanceOf[Partition]
   }
 
   override def run(attemptId: Long): MapStatus = {
     val numOutputSplits = dep.partitioner.numPartitions
-    val partitioner = dep.partitioner
 
     val taskContext = new TaskContext(stageId, partition, attemptId)
+    metrics = Some(taskContext.taskMetrics)
+    try {
+      // Partition the map output.
+      val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)])
+      for (elem <- rdd.iterator(split, taskContext)) {
+        val pair = elem.asInstanceOf[(Any, Any)]
+        val bucketId = dep.partitioner.getPartition(pair._1)
+        buckets(bucketId) += pair
+      }
 
-    // Partition the map output.
-    val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)])
-    for (elem <- rdd.iterator(split, taskContext)) {
-      val pair = elem.asInstanceOf[(Any, Any)]
-      val bucketId = partitioner.getPartition(pair._1)
-      buckets(bucketId) += pair
-    }
-    val bucketIterators = buckets.map(_.iterator)
+      val compressedSizes = new Array[Byte](numOutputSplits)
 
-    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)] = bucketIterators(i)
-      val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false)
-      compressedSizes(i) = MapOutputTracker.compressSize(size)
+      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)
+        totalBytes += size
+        compressedSizes(i) = MapOutputTracker.compressSize(size)
+      }
+      val shuffleMetrics = new ShuffleWriteMetrics
+      shuffleMetrics.shuffleBytesWritten = totalBytes
+      metrics.get.shuffleWriteMetrics = Some(shuffleMetrics)
+
+      return new MapStatus(blockManager.blockManagerId, compressedSizes)
+    } finally {
+      // Execute the callbacks on task completion.
+      taskContext.executeOnCompleteCallbacks()
     }
-
-    // Execute the callbacks on task completion.
-    taskContext.executeOnCompleteCallbacks()
-
-    return new MapStatus(blockManager.blockManagerId, compressedSizes)
   }
 
   override def preferredLocations: Seq[String] = locs
diff --git a/core/src/main/scala/spark/scheduler/SparkListener.scala b/core/src/main/scala/spark/scheduler/SparkListener.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a65140b145833c70e584936785ac8fe94d6b0bc2
--- /dev/null
+++ b/core/src/main/scala/spark/scheduler/SparkListener.scala
@@ -0,0 +1,146 @@
+package spark.scheduler
+
+import spark.scheduler.cluster.TaskInfo
+import spark.util.Distribution
+import spark.{Utils, Logging}
+import spark.executor.TaskMetrics
+
+trait SparkListener {
+  /**
+   * called when a stage is completed, with information on the completed stage
+   */
+  def onStageCompleted(stageCompleted: StageCompleted)
+}
+
+sealed trait SparkListenerEvents
+
+case class StageCompleted(val stageInfo: StageInfo) extends SparkListenerEvents
+
+
+/**
+ * Simple SparkListener that logs a few summary statistics when each stage completes
+ */
+class StatsReportListener extends SparkListener with Logging {
+  def onStageCompleted(stageCompleted: StageCompleted) {
+    import spark.scheduler.StatsReportListener._
+    implicit val sc = stageCompleted
+    this.logInfo("Finished stage: " + stageCompleted.stageInfo)
+    showMillisDistribution("task runtime:", (info, _) => Some(info.duration))
+
+    //shuffle write
+    showBytesDistribution("shuffle bytes written:",(_,metric) => metric.shuffleWriteMetrics.map{_.shuffleBytesWritten})
+
+    //fetch & io
+    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))
+
+    //runtime breakdown
+    val runtimePcts = stageCompleted.stageInfo.taskInfos.map{
+      case (info, metrics) => RuntimePercentage(info.duration, metrics)
+    }
+    showDistribution("executor (non-fetch) time pct: ", Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%")
+    showDistribution("fetch wait time pct: ", Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%")
+    showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%")
+  }
+
+}
+
+object StatsReportListener extends Logging {
+
+  //for profiling, the extremes are more interesting
+  val percentiles = Array[Int](0,5,10,25,50,75,90,95,100)
+  val probabilities = percentiles.map{_ / 100.0}
+  val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%"
+
+  def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = {
+    Distribution(stage.stageInfo.taskInfos.flatMap{
+      case ((info,metric)) => getMetric(info, metric)})
+  }
+
+  //is there some way to setup the types that I can get rid of this completely?
+  def extractLongDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]): Option[Distribution] = {
+    extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble})
+  }
+
+  def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) {
+    val stats = d.statCounter
+    logInfo(heading + stats)
+    val quantiles = d.getQuantiles(probabilities).map{formatNumber}
+    logInfo(percentilesHeader)
+    logInfo("\t" + quantiles.mkString("\t"))
+  }
+
+  def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) {
+    dOpt.foreach { d => showDistribution(heading, d, formatNumber)}
+  }
+
+  def showDistribution(heading: String, dOpt: Option[Distribution], format:String) {
+    def f(d:Double) = format.format(d)
+    showDistribution(heading, dOpt, f _)
+  }
+
+  def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double])
+    (implicit stage: StageCompleted) {
+    showDistribution(heading, extractDoubleDistribution(stage, getMetric), format)
+  }
+
+  def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long])
+    (implicit stage: StageCompleted) {
+    showBytesDistribution(heading, extractLongDistribution(stage, getMetric))
+  }
+
+  def showBytesDistribution(heading: String, dOpt: Option[Distribution]) {
+    dOpt.foreach{dist => showBytesDistribution(heading, dist)}
+  }
+
+  def showBytesDistribution(heading: String, dist: Distribution) {
+    showDistribution(heading, dist, (d => Utils.memoryBytesToString(d.toLong)): Double => String)
+  }
+
+  def showMillisDistribution(heading: String, dOpt: Option[Distribution]) {
+    showDistribution(heading, dOpt, (d => StatsReportListener.millisToString(d.toLong)): Double => String)
+  }
+
+  def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long])
+    (implicit stage: StageCompleted) {
+    showMillisDistribution(heading, extractLongDistribution(stage, getMetric))
+  }
+
+
+
+  val seconds = 1000L
+  val minutes = seconds * 60
+  val hours = minutes * 60
+
+  /**
+   * reformat a time interval in milliseconds to a prettier format for output
+   */
+  def millisToString(ms: Long) = {
+    val (size, units) =
+      if (ms > hours) {
+        (ms.toDouble / hours, "hours")
+      } else if (ms > minutes) {
+        (ms.toDouble / minutes, "min")
+      } else if (ms > seconds) {
+        (ms.toDouble / seconds, "s")
+      } else {
+        (ms.toDouble, "ms")
+      }
+    "%.1f %s".format(size, units)
+  }
+}
+
+
+
+case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double)
+object RuntimePercentage {
+  def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = {
+    val denom = totalTime.toDouble
+    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))
+    RuntimePercentage(exec, fetch, other)
+  }
+}
diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala
index 4846b6672930367e3b5983ecb5dc7f5399fdf1bd..552061e46b8c18d53ac06496a9b27c8cc37c5b80 100644
--- a/core/src/main/scala/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/spark/scheduler/Stage.scala
@@ -28,10 +28,13 @@ private[spark] class Stage(
   extends Logging {
   
   val isShuffleMap = shuffleDep != None
-  val numPartitions = rdd.splits.size
+  val numPartitions = rdd.partitions.size
   val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil)
   var numAvailableOutputs = 0
 
+  /** When first task was submitted to scheduler. */
+  var submissionTime: Option[Long] = None
+
   private var nextAttemptId = 0
 
   def isAvailable: Boolean = {
@@ -51,18 +54,18 @@ private[spark] class Stage(
 
   def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) {
     val prevList = outputLocs(partition)
-    val newList = prevList.filterNot(_.address == bmAddress)
+    val newList = prevList.filterNot(_.location == bmAddress)
     outputLocs(partition) = newList
     if (prevList != Nil && newList == Nil) {
       numAvailableOutputs -= 1
     }
   }
  
-  def removeOutputsOnHost(host: String) {
+  def removeOutputsOnExecutor(execId: String) {
     var becameUnavailable = false
     for (partition <- 0 until numPartitions) {
       val prevList = outputLocs(partition)
-      val newList = prevList.filterNot(_.address.ip == host)
+      val newList = prevList.filterNot(_.location.executorId == execId)
       outputLocs(partition) = newList
       if (prevList != Nil && newList == Nil) {
         becameUnavailable = true
@@ -70,7 +73,8 @@ private[spark] class Stage(
       }
     }
     if (becameUnavailable) {
-      logInfo("%s is now unavailable on %s (%d/%d, %s)".format(this, host, numAvailableOutputs, numPartitions, isAvailable))
+      logInfo("%s is now unavailable on executor %s (%d/%d, %s)".format(
+        this, execId, numAvailableOutputs, numPartitions, isAvailable))
     }
   }
 
@@ -82,7 +86,7 @@ private[spark] class Stage(
 
   def origin: String = rdd.origin
 
-  override def toString = "Stage " + id // + ": [RDD = " + rdd.id + ", isShuffle = " + isShuffleMap + "]"
+  override def toString = "Stage " + id
 
   override def hashCode(): Int = id
 }
diff --git a/core/src/main/scala/spark/scheduler/StageInfo.scala b/core/src/main/scala/spark/scheduler/StageInfo.scala
new file mode 100644
index 0000000000000000000000000000000000000000..8d83ff10c420d97bd23697ceee64774e8dd93bb3
--- /dev/null
+++ b/core/src/main/scala/spark/scheduler/StageInfo.scala
@@ -0,0 +1,12 @@
+package spark.scheduler
+
+import spark.scheduler.cluster.TaskInfo
+import scala.collection._
+import spark.executor.TaskMetrics
+
+case class StageInfo(
+    val stage: Stage,
+    val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = mutable.Buffer[(TaskInfo, TaskMetrics)]()
+) {
+  override def toString = stage.rdd.toString
+}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala
index ef987fdeb696bbe96f11868ad6b8930cf2331f41..a6462c6968b67492399304c2e4d7887986e349ac 100644
--- a/core/src/main/scala/spark/scheduler/Task.scala
+++ b/core/src/main/scala/spark/scheduler/Task.scala
@@ -1,12 +1,12 @@
 package spark.scheduler
 
-import scala.collection.mutable.HashMap
-import spark.serializer.{SerializerInstance, Serializer}
+import spark.serializer.SerializerInstance
 import java.io.{DataInputStream, DataOutputStream}
 import java.nio.ByteBuffer
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 import spark.util.ByteBufferInputStream
 import scala.collection.mutable.HashMap
+import spark.executor.TaskMetrics
 
 /**
  * A task to execute on a worker node.
@@ -16,6 +16,9 @@ private[spark] abstract class Task[T](val stageId: Int) extends Serializable {
   def preferredLocations: Seq[String] = Nil
 
   var generation: Long = -1   // Map output tracker generation. Will be set by TaskScheduler.
+
+  var metrics: Option[TaskMetrics] = None
+
 }
 
 /**
diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala
index 9a54d0e8541eb46381b2387e2d578222f60c0d6a..6de0aa7adf8ac782895b53d568286e2f2c4a1cdf 100644
--- a/core/src/main/scala/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/spark/scheduler/TaskResult.scala
@@ -3,13 +3,14 @@ package spark.scheduler
 import java.io._
 
 import scala.collection.mutable.Map
+import spark.executor.TaskMetrics
 
 // Task result. Also contains updates to accumulator variables.
 // TODO: Use of distributed cache to return result is a hack to get around
 // what seems to be a bug with messages over 60KB in libprocess; fix it
 private[spark]
-class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Externalizable {
-  def this() = this(null.asInstanceOf[T], null)
+class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends Externalizable {
+  def this() = this(null.asInstanceOf[T], null, null)
 
   override def writeExternal(out: ObjectOutput) {
     out.writeObject(value)
@@ -18,6 +19,7 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Exte
       out.writeLong(key)
       out.writeObject(value)
     }
+    out.writeObject(metrics)
   }
 
   override def readExternal(in: ObjectInput) {
@@ -31,5 +33,6 @@ class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Exte
         accumUpdates(in.readLong()) = in.readObject()
       }
     }
+    metrics = in.readObject().asInstanceOf[TaskMetrics]
   }
 }
diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
index fa4de15d0da605c8252f26163af0374fe9a8746d..771518dddfacaaf2916a1f6cd834983725cbc533 100644
--- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
+++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
@@ -1,18 +1,21 @@
 package spark.scheduler
 
+import spark.scheduler.cluster.TaskInfo
 import scala.collection.mutable.Map
 
 import spark.TaskEndReason
+import spark.executor.TaskMetrics
 
 /**
  * Interface for getting events back from the TaskScheduler.
  */
 private[spark] trait TaskSchedulerListener {
   // A task has finished or failed.
-  def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]): Unit
+  def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any],
+                taskInfo: TaskInfo, taskMetrics: TaskMetrics): Unit
 
   // A node was lost from the cluster.
-  def hostLost(host: String): Unit
+  def executorLost(execId: String): Unit
 
   // The TaskScheduler wants to abort an entire task set.
   def taskSetFailed(taskSet: TaskSet, reason: String): Unit
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
index 20f6e65020f54d87418ffb8a1441a886f759c754..26fdef101bb278ecd363b1c281995d0b8f3256fc 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
@@ -11,6 +11,7 @@ import spark.TaskState.TaskState
 import spark.scheduler._
 import java.nio.ByteBuffer
 import java.util.concurrent.atomic.AtomicLong
+import java.util.{TimerTask, Timer}
 
 /**
  * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call
@@ -22,24 +23,31 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
 
   // How often to check for speculative tasks
   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
 
   val activeTaskSets = new HashMap[String, TaskSetManager]
   var activeTaskSetsQueue = new ArrayBuffer[TaskSetManager]
 
   val taskIdToTaskSetId = new HashMap[Long, String]
-  val taskIdToSlaveId = new HashMap[Long, String]
+  val taskIdToExecutorId = new HashMap[Long, String]
   val taskSetTaskIds = new HashMap[String, HashSet[Long]]
 
+  var hasReceivedTask = false
+  var hasLaunchedTask = false
+  val starvationTimer = new Timer(true)
+
   // Incrementing Mesos task IDs
   val nextTaskId = new AtomicLong(0)
 
-  // Which hosts in the cluster are alive (contains hostnames)
-  val hostsAlive = new HashSet[String]
+  // Which executor IDs we have executors on
+  val activeExecutorIds = new HashSet[String]
 
-  // Which slave IDs we have executors on
-  val slaveIdsWithExecutors = new 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]]
 
-  val slaveIdToHost = new HashMap[String, String]
+  val executorIdToHost = new HashMap[String, String]
 
   // JAR server, if any JARs were added by the user to the SparkContext
   var jarServer: HttpServer = null
@@ -85,7 +93,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     }
   }
 
-  def submitTasks(taskSet: TaskSet) {
+  override def submitTasks(taskSet: TaskSet) {
     val tasks = taskSet.tasks
     logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks")
     this.synchronized {
@@ -93,6 +101,20 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
       activeTaskSets(taskSet.id) = manager
       activeTaskSetsQueue += manager
       taskSetTaskIds(taskSet.id) = new HashSet[Long]()
+
+      if (hasReceivedTask == false) {
+        starvationTimer.scheduleAtFixedRate(new TimerTask() {
+          override def run() {
+            if (!hasLaunchedTask) {
+              logWarning("Initial job has not accepted any resources; " +
+                "check your cluster UI to ensure that workers are registered")
+            } else {
+              this.cancel()
+            }
+          }
+        }, STARVATION_TIMEOUT, STARVATION_TIMEOUT)
+      }
+      hasReceivedTask = true;
     }
     backend.reviveOffers()
   }
@@ -102,7 +124,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
       activeTaskSets -= manager.taskSet.id
       activeTaskSetsQueue -= manager
       taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id)
-      taskIdToSlaveId --= taskSetTaskIds(manager.taskSet.id)
+      taskIdToExecutorId --= taskSetTaskIds(manager.taskSet.id)
       taskSetTaskIds.remove(manager.taskSet.id)
     }
   }
@@ -117,8 +139,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
       SparkEnv.set(sc.env)
       // Mark each slave as alive and remember its hostname
       for (o <- offers) {
-        slaveIdToHost(o.slaveId) = o.hostname
-        hostsAlive += o.hostname
+        executorIdToHost(o.executorId) = o.hostname
+        if (!executorsByHost.contains(o.hostname)) {
+          executorsByHost(o.hostname) = new HashSet()
+        }
       }
       // Build a list of tasks to assign to each slave
       val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
@@ -128,16 +152,17 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
         do {
           launchedTask = false
           for (i <- 0 until offers.size) {
-            val sid = offers(i).slaveId
+            val execId = offers(i).executorId
             val host = offers(i).hostname
-            manager.slaveOffer(sid, host, availableCpus(i)) match {
+            manager.slaveOffer(execId, host, availableCpus(i)) match {
               case Some(task) =>
                 tasks(i) += task
                 val tid = task.taskId
                 taskIdToTaskSetId(tid) = manager.taskSet.id
                 taskSetTaskIds(manager.taskSet.id) += tid
-                taskIdToSlaveId(tid) = sid
-                slaveIdsWithExecutors += sid
+                taskIdToExecutorId(tid) = execId
+                activeExecutorIds += execId
+                executorsByHost(host) += execId
                 availableCpus(i) -= 1
                 launchedTask = true
 
@@ -146,31 +171,30 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
           }
         } while (launchedTask)
       }
+      if (tasks.size > 0) {
+        hasLaunchedTask = true
+      }
       return tasks
     }
   }
 
   def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
     var taskSetToUpdate: Option[TaskSetManager] = None
-    var failedHost: Option[String] = None
+    var failedExecutor: Option[String] = None
     var taskFailed = false
     synchronized {
       try {
-        if (state == TaskState.LOST && taskIdToSlaveId.contains(tid)) {
-          // We lost the executor on this slave, so remember that it's gone
-          val slaveId = taskIdToSlaveId(tid)
-          val host = slaveIdToHost(slaveId)
-          if (hostsAlive.contains(host)) {
-            slaveIdsWithExecutors -= slaveId
-            hostsAlive -= host
-            activeTaskSetsQueue.foreach(_.hostLost(host))
-            failedHost = Some(host)
+        if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) {
+          // We lost this entire executor, so remember that it's gone
+          val execId = taskIdToExecutorId(tid)
+          if (activeExecutorIds.contains(execId)) {
+            removeExecutor(execId)
+            failedExecutor = Some(execId)
           }
         }
         taskIdToTaskSetId.get(tid) match {
           case Some(taskSetId) =>
             if (activeTaskSets.contains(taskSetId)) {
-              //activeTaskSets(taskSetId).statusUpdate(status)
               taskSetToUpdate = Some(activeTaskSets(taskSetId))
             }
             if (TaskState.isFinished(state)) {
@@ -178,7 +202,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
               if (taskSetTaskIds.contains(taskSetId)) {
                 taskSetTaskIds(taskSetId) -= tid
               }
-              taskIdToSlaveId.remove(tid)
+              taskIdToExecutorId.remove(tid)
             }
             if (state == TaskState.FAILED) {
               taskFailed = true
@@ -190,12 +214,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
         case e: Exception => logError("Exception in statusUpdate", e)
       }
     }
-    // Update the task set and DAGScheduler without holding a lock on this, because that can deadlock
+    // Update the task set and DAGScheduler without holding a lock on this, since that can deadlock
     if (taskSetToUpdate != None) {
       taskSetToUpdate.get.statusUpdate(tid, state, serializedData)
     }
-    if (failedHost != None) {
-      listener.hostLost(failedHost.get)
+    if (failedExecutor != None) {
+      listener.executorLost(failedExecutor.get)
       backend.reviveOffers()
     }
     if (taskFailed) {
@@ -235,7 +259,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   }
 
   override def defaultParallelism() = backend.defaultParallelism()
-  
+
   // Check for speculatable tasks in all our active jobs.
   def checkSpeculatableTasks() {
     var shouldRevive = false
@@ -249,27 +273,42 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
     }
   }
 
-  def slaveLost(slaveId: String, reason: ExecutorLossReason) {
-    var failedHost: Option[String] = None
+  def executorLost(executorId: String, reason: ExecutorLossReason) {
+    var failedExecutor: Option[String] = None
     synchronized {
-      val host = slaveIdToHost(slaveId)
-      if (hostsAlive.contains(host)) {
-        logError("Lost an executor on " + host + ": " + reason)
-        slaveIdsWithExecutors -= slaveId
-        hostsAlive -= host
-        activeTaskSetsQueue.foreach(_.hostLost(host))
-        failedHost = Some(host)
+      if (activeExecutorIds.contains(executorId)) {
+        val host = executorIdToHost(executorId)
+        logError("Lost executor %s on %s: %s".format(executorId, host, reason))
+        removeExecutor(executorId)
+        failedExecutor = Some(executorId)
       } else {
-        // We may get multiple slaveLost() calls with different loss reasons. For example, one 
-        // may be triggered by a dropped connection from the slave while another may be a report
-        // of executor termination from Mesos. We produce log messages for both so we eventually
-        // report the termination reason.
-        logError("Lost an executor on " + host + " (already removed): " + reason)
+         // We may get multiple executorLost() calls with different loss reasons. For example, one
+         // may be triggered by a dropped connection from the slave while another may be a report
+         // of executor termination from Mesos. We produce log messages for both so we eventually
+         // report the termination reason.
+         logError("Lost an executor " + executorId + " (already removed): " + reason)
       }
     }
-    if (failedHost != None) {
-      listener.hostLost(failedHost.get)
+    // Call listener.executorLost without holding the lock on this to prevent deadlock
+    if (failedExecutor != None) {
+      listener.executorLost(failedExecutor.get)
       backend.reviveOffers()
     }
   }
+
+  /** 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)
+    execs -= executorId
+    if (execs.isEmpty) {
+      executorsByHost -= host
+    }
+    executorIdToHost -= executorId
+    activeTaskSetsQueue.foreach(_.executorLost(executorId, host))
+  }
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
index bba7de6a65c3d17aab47bdfa07c464ee7e801604..8bf838209f3d8183cf6251bd7def1b8f14dffa75 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ExecutorLossReason.scala
@@ -12,10 +12,10 @@ class ExecutorLossReason(val message: String) {
 
 private[spark]
 case class ExecutorExited(val exitCode: Int)
-    extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) {
+  extends ExecutorLossReason(ExecutorExitCode.explainExitCode(exitCode)) {
 }
 
 private[spark]
 case class SlaveLost(_message: String = "Slave lost")
-    extends ExecutorLossReason(_message) {
+  extends ExecutorLossReason(_message) {
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
index ddcd64d7c6d1f4576df611620037db2869746fe3..9ac875de3a16a890a5654c79581e15b7bd607ec4 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
@@ -1,5 +1,7 @@
 package spark.scheduler.cluster
 
+import spark.Utils
+
 /**
  * A backend interface for cluster scheduling systems that allows plugging in different ones under
  * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as
@@ -11,5 +13,15 @@ private[spark] trait SchedulerBackend {
   def reviveOffers(): Unit
   def defaultParallelism(): Int
 
+  // Memory used by each executor (in megabytes)
+  protected val executorMemory = {
+    // TODO: Might need to add some extra memory for the non-heap parts of the JVM
+    Option(System.getProperty("spark.executor.memory"))
+      .orElse(Option(System.getenv("SPARK_MEM")))
+      .map(Utils.memoryStringToMb)
+      .getOrElse(512)
+  }
+
+
   // TODO: Probably want to add a killTask too
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala b/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala
deleted file mode 100644
index 96ebaa460118e45a9dbd40cb3a5c0c36a0b13907..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala
+++ /dev/null
@@ -1,4 +0,0 @@
-package spark.scheduler.cluster
-
-private[spark]
-class SlaveResources(val slaveId: String, val hostname: String, val coresFree: Int) {}
diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index e2301347e510340b25c4d10731dc6db7987048c0..bb289c9cf391b4d1dcc6f94113117b9468c4a578 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -2,14 +2,14 @@ package spark.scheduler.cluster
 
 import spark.{Utils, Logging, SparkContext}
 import spark.deploy.client.{Client, ClientListener}
-import spark.deploy.{Command, JobDescription}
+import spark.deploy.{Command, ApplicationDescription}
 import scala.collection.mutable.HashMap
 
 private[spark] class SparkDeploySchedulerBackend(
     scheduler: ClusterScheduler,
     sc: SparkContext,
     master: String,
-    jobName: String)
+    appName: String)
   extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
   with ClientListener
   with Logging {
@@ -19,34 +19,26 @@ private[spark] class SparkDeploySchedulerBackend(
   var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
 
   val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
-  val executorIdToSlaveId = new HashMap[String, String]
-
-  // Memory used by each executor (in megabytes)
-  val executorMemory = {
-    if (System.getenv("SPARK_MEM") != null) {
-      Utils.memoryStringToMb(System.getenv("SPARK_MEM"))
-      // TODO: Might need to add some extra memory for the non-heap parts of the JVM
-    } else {
-      512
-    }
-  }
 
   override def start() {
     super.start()
 
-    val masterUrl = "akka://spark@%s:%s/user/%s".format(
-      System.getProperty("spark.master.host"), System.getProperty("spark.master.port"),
+    // The endpoint for executors to talk to us
+    val driverUrl = "akka://spark@%s:%s/user/%s".format(
+      System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
       StandaloneSchedulerBackend.ACTOR_NAME)
-    val args = Seq(masterUrl, "{{SLAVEID}}", "{{HOSTNAME}}", "{{CORES}}")
+    val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
     val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
-    val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command)
+    val sparkHome = sc.getSparkHome().getOrElse(
+      throw new IllegalArgumentException("must supply spark home for spark standalone"))
+    val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome)
 
-    client = new Client(sc.env.actorSystem, master, jobDesc, this)
+    client = new Client(sc.env.actorSystem, master, appDesc, this)
     client.start()
   }
 
   override def stop() {
-    stopping = true;
+    stopping = true
     super.stop()
     client.stop()
     if (shutdownCallback != null) {
@@ -54,35 +46,28 @@ private[spark] class SparkDeploySchedulerBackend(
     }
   }
 
-  def connected(jobId: String) {
-    logInfo("Connected to Spark cluster with job ID " + jobId)
+  override def connected(appId: String) {
+    logInfo("Connected to Spark cluster with app ID " + appId)
   }
 
-  def disconnected() {
+  override def disconnected() {
     if (!stopping) {
       logError("Disconnected from Spark cluster!")
       scheduler.error("Disconnected from Spark cluster")
     }
   }
 
-  def executorAdded(id: String, workerId: String, host: String, cores: Int, memory: Int) {
-    executorIdToSlaveId += id -> workerId
+  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(
-       id, host, cores, Utils.memoryMegabytesToString(memory)))
+       executorId, host, cores, Utils.memoryMegabytesToString(memory)))
   }
 
-  def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {
+  override def executorRemoved(executorId: String, message: String, exitStatus: Option[Int]) {
     val reason: ExecutorLossReason = exitStatus match {
       case Some(code) => ExecutorExited(code)
       case None => SlaveLost(message)
     }
-    logInfo("Executor %s removed: %s".format(id, message))
-    executorIdToSlaveId.get(id) match {
-      case Some(slaveId) => 
-        executorIdToSlaveId.remove(id)
-        scheduler.slaveLost(slaveId, reason)
-      case None =>
-        logInfo("No slave ID known for executor %s".format(id))
-    }
+    logInfo("Executor %s removed: %s".format(executorId, message))
+    removeExecutor(executorId, reason.toString)
   }
 }
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
index 1386cd9d4402e68b60379caf34144d5af17fd9c3..d7660678248b2d9f028d8364bd3caa8cbd47660c 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
@@ -6,32 +6,37 @@ import spark.util.SerializableBuffer
 
 private[spark] sealed trait StandaloneClusterMessage extends Serializable
 
-// Master to slaves
+// Driver to executors
 private[spark]
 case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage
 
 private[spark]
-case class RegisteredSlave(sparkProperties: Seq[(String, String)]) extends StandaloneClusterMessage
+case class RegisteredExecutor(sparkProperties: Seq[(String, String)])
+  extends StandaloneClusterMessage
 
 private[spark]
-case class RegisterSlaveFailed(message: String) extends StandaloneClusterMessage
+case class RegisterExecutorFailed(message: String) extends StandaloneClusterMessage
 
-// Slaves to master
+// Executors to driver
 private[spark]
-case class RegisterSlave(slaveId: String, host: String, cores: Int) extends StandaloneClusterMessage
+case class RegisterExecutor(executorId: String, host: String, cores: Int)
+  extends StandaloneClusterMessage
 
 private[spark]
-case class StatusUpdate(slaveId: String, taskId: Long, state: TaskState, data: SerializableBuffer)
+case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, data: SerializableBuffer)
   extends StandaloneClusterMessage
 
 private[spark]
 object StatusUpdate {
   /** Alternate factory method that takes a ByteBuffer directly for the data field */
-  def apply(slaveId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = {
-    StatusUpdate(slaveId, taskId, state, new SerializableBuffer(data))
+  def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = {
+    StatusUpdate(executorId, taskId, state, new SerializableBuffer(data))
   }
 }
 
-// Internal messages in master
+// Internal messages in driver
 private[spark] case object ReviveOffers extends StandaloneClusterMessage
-private[spark] case object StopMaster extends StandaloneClusterMessage
+private[spark] case object StopDriver extends StandaloneClusterMessage
+
+private[spark] case class RemoveExecutor(executorId: String, reason: String)
+  extends StandaloneClusterMessage
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 03dc5f4b9b4abff52968c22fd74a52c014cd21c6..a06d853b466dfe5ecb0ee63aa71fc4e2e8bfa3f9 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -23,13 +23,13 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
   // Use an atomic variable to track total number of cores in the cluster for simplicity and speed
   var totalCoreCount = new AtomicInteger(0)
 
-  class MasterActor(sparkProperties: Seq[(String, String)]) extends Actor {
-    val slaveActor = new HashMap[String, ActorRef]
-    val slaveAddress = new HashMap[String, Address]
-    val slaveHost = new HashMap[String, String]
+  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 actorToSlaveId = new HashMap[ActorRef, String]
-    val addressToSlaveId = new HashMap[Address, String]
+    val actorToExecutorId = new HashMap[ActorRef, String]
+    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,86 +37,92 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     }
 
     def receive = {
-      case RegisterSlave(slaveId, host, cores) =>
-        if (slaveActor.contains(slaveId)) {
-          sender ! RegisterSlaveFailed("Duplicate slave ID: " + slaveId)
+      case RegisterExecutor(executorId, host, cores) =>
+        if (executorActor.contains(executorId)) {
+          sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId)
         } else {
-          logInfo("Registered slave: " + sender + " with ID " + slaveId)
-          sender ! RegisteredSlave(sparkProperties)
+          logInfo("Registered executor: " + sender + " with ID " + executorId)
+          sender ! RegisteredExecutor(sparkProperties)
           context.watch(sender)
-          slaveActor(slaveId) = sender
-          slaveHost(slaveId) = host
-          freeCores(slaveId) = cores
-          slaveAddress(slaveId) = sender.path.address
-          actorToSlaveId(sender) = slaveId
-          addressToSlaveId(sender.path.address) = slaveId
+          executorActor(executorId) = sender
+          executorHost(executorId) = host
+          freeCores(executorId) = cores
+          executorAddress(executorId) = sender.path.address
+          actorToExecutorId(sender) = executorId
+          addressToExecutorId(sender.path.address) = executorId
           totalCoreCount.addAndGet(cores)
           makeOffers()
         }
 
-      case StatusUpdate(slaveId, taskId, state, data) =>
+      case StatusUpdate(executorId, taskId, state, data) =>
         scheduler.statusUpdate(taskId, state, data.value)
         if (TaskState.isFinished(state)) {
-          freeCores(slaveId) += 1
-          makeOffers(slaveId)
+          freeCores(executorId) += 1
+          makeOffers(executorId)
         }
 
       case ReviveOffers =>
         makeOffers()
 
-      case StopMaster =>
+      case StopDriver =>
         sender ! true
         context.stop(self)
 
+      case RemoveExecutor(executorId, reason) =>
+        removeExecutor(executorId, reason)
+        sender ! true
+
       case Terminated(actor) =>
-        actorToSlaveId.get(actor).foreach(removeSlave(_, "Akka actor terminated"))
+        actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
 
       case RemoteClientDisconnected(transport, address) =>
-        addressToSlaveId.get(address).foreach(removeSlave(_, "remote Akka client disconnected"))
+        addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected"))
 
       case RemoteClientShutdown(transport, address) =>
-        addressToSlaveId.get(address).foreach(removeSlave(_, "remote Akka client shutdown"))
+        addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown"))
     }
 
-    // Make fake resource offers on all slaves
+    // Make fake resource offers on all executors
     def makeOffers() {
       launchTasks(scheduler.resourceOffers(
-        slaveHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))}))
+        executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))}))
     }
 
-    // Make fake resource offers on just one slave
-    def makeOffers(slaveId: String) {
+    // Make fake resource offers on just one executor
+    def makeOffers(executorId: String) {
       launchTasks(scheduler.resourceOffers(
-        Seq(new WorkerOffer(slaveId, slaveHost(slaveId), freeCores(slaveId)))))
+        Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId)))))
     }
 
     // Launch tasks returned by a set of resource offers
     def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
       for (task <- tasks.flatten) {
-        freeCores(task.slaveId) -= 1
-        slaveActor(task.slaveId) ! LaunchTask(task)
+        freeCores(task.executorId) -= 1
+        executorActor(task.executorId) ! LaunchTask(task)
       }
     }
 
     // Remove a disconnected slave from the cluster
-    def removeSlave(slaveId: String, reason: String) {
-      logInfo("Slave " + slaveId + " disconnected, so removing it")
-      val numCores = freeCores(slaveId)
-      actorToSlaveId -= slaveActor(slaveId)
-      addressToSlaveId -= slaveAddress(slaveId)
-      slaveActor -= slaveId
-      slaveHost -= slaveId
-      freeCores -= slaveId
-      slaveHost -= slaveId
-      totalCoreCount.addAndGet(-numCores)
-      scheduler.slaveLost(slaveId, SlaveLost(reason))
+    def removeExecutor(executorId: String, reason: String) {
+      if (executorActor.contains(executorId)) {
+        logInfo("Executor " + executorId + " disconnected, so removing it")
+        val numCores = freeCores(executorId)
+        actorToExecutorId -= executorActor(executorId)
+        addressToExecutorId -= executorAddress(executorId)
+        executorActor -= executorId
+        executorHost -= executorId
+        freeCores -= executorId
+        executorHost -= executorId
+        totalCoreCount.addAndGet(-numCores)
+        scheduler.executorLost(executorId, SlaveLost(reason))
+      }
     }
   }
 
-  var masterActor: ActorRef = null
+  var driverActor: ActorRef = null
   val taskIdsOnSlave = new HashMap[String, HashSet[String]]
 
-  def start() {
+  override def start() {
     val properties = new ArrayBuffer[(String, String)]
     val iterator = System.getProperties.entrySet.iterator
     while (iterator.hasNext) {
@@ -126,28 +132,41 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
         properties += ((key, value))
       }
     }
-    masterActor = actorSystem.actorOf(
-      Props(new MasterActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME)
+    driverActor = actorSystem.actorOf(
+      Props(new DriverActor(properties)), name = StandaloneSchedulerBackend.ACTOR_NAME)
   }
 
-  def stop() {
+  override def stop() {
     try {
-      if (masterActor != null) {
+      if (driverActor != null) {
         val timeout = 5.seconds
-        val future = masterActor.ask(StopMaster)(timeout)
+        val future = driverActor.ask(StopDriver)(timeout)
         Await.result(future, timeout)
       }
     } catch {
       case e: Exception =>
-        throw new SparkException("Error stopping standalone scheduler's master actor", e)
+        throw new SparkException("Error stopping standalone scheduler's driver actor", e)
     }
   }
 
-  def reviveOffers() {
-    masterActor ! ReviveOffers
+  override def reviveOffers() {
+    driverActor ! ReviveOffers
   }
 
-  def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2)
+  override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism"))
+      .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
+
+  // 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 {
+      case e: Exception =>
+        throw new SparkException("Error notifying standalone scheduler's driver actor", e)
+    }
+  }
 }
 
 private[spark] object StandaloneSchedulerBackend {
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
index aa097fd3a25c1e89fd6793237aa134fb47d55e88..b41e951be99f84c68b800a1cf7a46636d3ced3f8 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
@@ -5,7 +5,7 @@ import spark.util.SerializableBuffer
 
 private[spark] class TaskDescription(
     val taskId: Long,
-    val slaveId: String,
+    val executorId: String,
     val name: String,
     _serializedTask: ByteBuffer)
   extends Serializable {
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
index ca845037801619788be9878a5c4d26c3f9243d4f..dfe3c5a85bc25f47a85a0da31a4649b27ef2c3be 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
@@ -4,7 +4,13 @@ package spark.scheduler.cluster
  * Information about a running task attempt inside a TaskSet.
  */
 private[spark]
-class TaskInfo(val taskId: Long, val index: Int, val launchTime: Long, val host: String) {
+class TaskInfo(
+    val taskId: Long,
+    val index: Int,
+    val launchTime: Long,
+    val executorId: String,
+    val host: String,
+    val preferred: Boolean) {
   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 a089b71644d3b643f5c6497e02a010998569b20b..c9f2c488048ca2628387165ac498d7346da45627 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
@@ -17,10 +17,7 @@ import java.nio.ByteBuffer
 /**
  * Schedules the tasks within a single TaskSet in the ClusterScheduler.
  */
-private[spark] class TaskSetManager(
-  sched: ClusterScheduler,
-  val taskSet: TaskSet)
-  extends Logging {
+private[spark] class TaskSetManager(sched: ClusterScheduler, val taskSet: TaskSet) extends Logging {
 
   // Maximum time to wait to run a task in a preferred location (in ms)
   val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong
@@ -100,7 +97,7 @@ private[spark] class TaskSetManager(
   }
 
   // Add a task to all the pending-task lists that it should be on.
-  def addPendingTask(index: Int) {
+  private def addPendingTask(index: Int) {
     val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive
     if (locations.size == 0) {
       pendingTasksWithNoPrefs += index
@@ -115,7 +112,7 @@ private[spark] class TaskSetManager(
 
   // Return the pending tasks list for a given host, or an empty list if
   // there is no map entry for that host
-  def getPendingTasksForHost(host: String): ArrayBuffer[Int] = {
+  private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = {
     pendingTasksForHost.getOrElse(host, ArrayBuffer())
   }
 
@@ -123,7 +120,7 @@ private[spark] class TaskSetManager(
   // Return None if the list is empty.
   // This method also cleans up any tasks in the list that have already
   // been launched, since we want that to happen lazily.
-  def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = {
+  private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = {
     while (!list.isEmpty) {
       val index = list.last
       list.trimEnd(1)
@@ -137,11 +134,12 @@ 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).
-  def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = {
+  private def findSpeculativeTask(host: String, localOnly: Boolean): Option[Int] = {
+    val hostsAlive = sched.hostsAlive
     speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set
     val localTask = speculatableTasks.find {
         index =>
-          val locations = tasks(index).preferredLocations.toSet & sched.hostsAlive
+          val locations = tasks(index).preferredLocations.toSet & hostsAlive
           val attemptLocs = taskAttempts(index).map(_.host)
           (locations.size == 0 || locations.contains(host)) && !attemptLocs.contains(host)
       }
@@ -161,7 +159,7 @@ 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.
-  def findTask(host: String, localOnly: Boolean): Option[Int] = {
+  private def findTask(host: String, localOnly: Boolean): Option[Int] = {
     val localTask = findTaskFromList(getPendingTasksForHost(host))
     if (localTask != None) {
       return localTask
@@ -183,13 +181,13 @@ private[spark] class TaskSetManager(
   // 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
   // no preferred locations (in which we still count the launch as preferred).
-  def isPreferredLocation(task: Task[_], host: String): Boolean = {
+  private def isPreferredLocation(task: Task[_], host: String): Boolean = {
     val locs = task.preferredLocations
     return (locs.contains(host) || locs.isEmpty)
   }
 
   // Respond to an offer of a single slave from the scheduler by finding a task
-  def slaveOffer(slaveId: String, host: String, availableCpus: Double): Option[TaskDescription] = {
+  def slaveOffer(execId: String, host: String, availableCpus: Double): Option[TaskDescription] = {
     if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) {
       val time = System.currentTimeMillis
       val localOnly = (time - lastPreferredLaunchTime < LOCALITY_WAIT)
@@ -206,11 +204,11 @@ private[spark] class TaskSetManager(
           } else {
             "non-preferred, not one of " + task.preferredLocations.mkString(", ")
           }
-          logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format(
-            taskSet.id, index, taskId, slaveId, host, prefStr))
+          logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format(
+            taskSet.id, index, taskId, execId, host, prefStr))
           // Do various bookkeeping
           copiesRunning(index) += 1
-          val info = new TaskInfo(taskId, index, time, host)
+          val info = new TaskInfo(taskId, index, time, execId, host, preferred)
           taskInfos(taskId) = info
           taskAttempts(index) = info :: taskAttempts(index)
           if (preferred) {
@@ -224,7 +222,7 @@ private[spark] class TaskSetManager(
           logInfo("Serialized task %s:%d as %d bytes in %d ms".format(
             taskSet.id, index, serializedTask.limit, timeTaken))
           val taskName = "task %s:%d".format(taskSet.id, index)
-          return Some(new TaskDescription(taskId, slaveId, taskName, serializedTask))
+          return Some(new TaskDescription(taskId, execId, taskName, serializedTask))
         }
         case _ =>
       }
@@ -261,7 +259,8 @@ private[spark] class TaskSetManager(
         tid, info.duration, tasksFinished, numTasks))
       // Deserialize task result and pass it to the scheduler
       val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader)
-      sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates)
+      result.metrics.resultSize = serializedData.limit()
+      sched.listener.taskEnded(tasks(index), Success, result.value, result.accumUpdates, info, result.metrics)
       // Mark finished and stop if we've finished all the tasks
       finished(index) = true
       if (tasksFinished == numTasks) {
@@ -292,7 +291,7 @@ private[spark] class TaskSetManager(
         reason match {
           case fetchFailed: FetchFailed =>
             logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress)
-            sched.listener.taskEnded(tasks(index), fetchFailed, null, null)
+            sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null)
             finished(index) = true
             tasksFinished += 1
             sched.taskSetFinished(this)
@@ -334,7 +333,7 @@ private[spark] class TaskSetManager(
         if (numFailures(index) > MAX_TASK_FAILURES) {
           logError("Task %s:%d failed more than %d times; aborting job".format(
             taskSet.id, index, MAX_TASK_FAILURES))
-          abort("Task %d failed more than %d times".format(index, MAX_TASK_FAILURES))
+          abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES))
         }
       }
     } else {
@@ -356,19 +355,22 @@ private[spark] class TaskSetManager(
     sched.taskSetFinished(this)
   }
 
-  def hostLost(hostname: String) {
-    logInfo("Re-queueing tasks for " + hostname + " from TaskSet " + taskSet.id)
-    // If some task has preferred locations only on hostname, put it in the no-prefs list
-    // to avoid the wait from delay scheduling
-    for (index <- getPendingTasksForHost(hostname)) {
-      val newLocs = tasks(index).preferredLocations.toSet & sched.hostsAlive
-      if (newLocs.isEmpty) {
-        pendingTasksWithNoPrefs += index
+  def executorLost(execId: String, hostname: 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
+        }
       }
     }
-    // Re-enqueue any tasks that ran on the failed host if this is a shuffle map stage
+    // 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.host == hostname) {
+      for ((tid, info) <- taskInfos if info.executorId == execId) {
         val index = taskInfos(tid).index
         if (finished(index)) {
           finished(index) = false
@@ -377,12 +379,12 @@ private[spark] class TaskSetManager(
           addPendingTask(index)
           // Tell the DAGScheduler that this task was resubmitted so that it doesn't think our
           // stage finishes when a total of tasks.size tasks finish.
-          sched.listener.taskEnded(tasks(index), Resubmitted, null, null)
+          sched.listener.taskEnded(tasks(index), Resubmitted, null, null, info, null)
         }
       }
     }
     // Also re-enqueue any tasks that were running on the node
-    for ((tid, info) <- taskInfos if info.running && info.host == hostname) {
+    for ((tid, info) <- taskInfos if info.running && info.executorId == execId) {
       taskLost(tid, TaskState.KILLED, null)
     }
   }
diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
index 6b919d68b26e677005fb6f17a9d233c4ce19d628..3c3afcbb14d3f7f677be8aae91128e35eef01c7f 100644
--- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
@@ -1,8 +1,8 @@
 package spark.scheduler.cluster
 
 /**
- * Represents free resources available on a worker node.
+ * Represents free resources available on an executor.
  */
 private[spark]
-class WorkerOffer(val slaveId: String, val hostname: String, val cores: Int) {
+class WorkerOffer(val executorId: String, val hostname: 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 2593c0e3a0d239e843b295f74761f23e18c39af9..9e1bde3fbe44e7196f5e3bb46b4edd71729e8f4b 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
@@ -1,14 +1,13 @@
 package spark.scheduler.local
 
 import java.io.File
-import java.net.URLClassLoader
-import java.util.concurrent.Executors
 import java.util.concurrent.atomic.AtomicInteger
 import scala.collection.mutable.HashMap
 
 import spark._
-import executor.ExecutorURLClassLoader
+import spark.executor.ExecutorURLClassLoader
 import spark.scheduler._
+import spark.scheduler.cluster.TaskInfo
 
 /**
  * A simple TaskScheduler implementation that runs tasks locally in a thread pool. Optionally
@@ -20,7 +19,7 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
   with Logging {
 
   var attemptId = new AtomicInteger(0)
-  var threadPool = Executors.newFixedThreadPool(threads, DaemonThreadFactory)
+  var threadPool = Utils.newDaemonFixedThreadPool(threads)
   val env = SparkEnv.get
   var listener: TaskSchedulerListener = null
 
@@ -53,7 +52,8 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
     }
 
     def runTask(task: Task[_], idInJob: Int, attemptId: Int) {
-      logInfo("Running task " + idInJob)
+      logInfo("Running " + task)
+      val info = new TaskInfo(attemptId, idInJob, System.currentTimeMillis(), "local", "local", true)
       // 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,11 +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 " + idInJob)
-        listener.taskEnded(task, Success, resultToReturn, accumUpdates)
+        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, deserializedTask.metrics.getOrElse(null))
       } catch {
         case t: Throwable => {
           logError("Exception in task " + idInJob, t)
@@ -91,7 +101,8 @@ 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
-              listener.taskEnded(task, new ExceptionFailure(t), null, null)
+              if (!Thread.currentThread().isInterrupted)
+                listener.taskEnded(task, new ExceptionFailure(t), null, null, info, null)
             }
           }
         }
@@ -112,16 +123,16 @@ private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkCon
       // Fetch missing dependencies
       for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
         logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File("."))
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
         currentFiles(name) = timestamp
       }
       for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
         logInfo("Fetching " + name + " with timestamp " + timestamp)
-        Utils.fetchFile(name, new File("."))
+        Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
         currentJars(name) = timestamp
         // Add it to our class loader
         val localName = name.split("/").last
-        val url = new File(".", localName).toURI.toURL
+        val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL
         if (!classLoader.getURLs.contains(url)) {
           logInfo("Adding " + url + " to class loader")
           classLoader.addURL(url)
diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index c45c7df69c63d797a24a2b05b8bcb8cf781e2f13..f4a2994b6d61d5b74ff8e331a2558f90fb4a8baf 100644
--- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -28,23 +28,13 @@ private[spark] class CoarseMesosSchedulerBackend(
     scheduler: ClusterScheduler,
     sc: SparkContext,
     master: String,
-    frameworkName: String)
+    appName: String)
   extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
   with MScheduler
   with Logging {
 
   val MAX_SLAVE_FAILURES = 2     // Blacklist a slave after this many failures
 
-  // Memory used by each executor (in megabytes)
-  val executorMemory = {
-    if (System.getenv("SPARK_MEM") != null) {
-      Utils.memoryStringToMb(System.getenv("SPARK_MEM"))
-      // TODO: Might need to add some extra memory for the non-heap parts of the JVM
-    } else {
-      512
-    }
-  }
-
   // Lock used to wait for scheduler to be registered
   var isRegistered = false
   val registeredLock = new Object()
@@ -64,13 +54,9 @@ private[spark] class CoarseMesosSchedulerBackend(
   val taskIdToSlaveId = new HashMap[Int, String]
   val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed
 
-  val sparkHome = sc.getSparkHome() match {
-    case Some(path) =>
-      path
-    case None =>
-      throw new SparkException("Spark home is not set; set it through the spark.home system " +
-        "property, the SPARK_HOME environment variable or the SparkContext constructor")
-  }
+  val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException(
+    "Spark home is not set; set it through the spark.home system " +
+    "property, the SPARK_HOME environment variable or the SparkContext constructor"))
 
   val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt
 
@@ -90,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend(
         setDaemon(true)
         override def run() {
           val scheduler = CoarseMesosSchedulerBackend.this
-          val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(frameworkName).build()
+          val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build()
           driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
           try { {
             val ret = driver.run()
@@ -108,11 +94,11 @@ private[spark] class CoarseMesosSchedulerBackend(
 
   def createCommand(offer: Offer, numCores: Int): CommandInfo = {
     val runScript = new File(sparkHome, "run").getCanonicalPath
-    val masterUrl = "akka://spark@%s:%s/user/%s".format(
-      System.getProperty("spark.master.host"), System.getProperty("spark.master.port"),
+    val driverUrl = "akka://spark@%s:%s/user/%s".format(
+      System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
       StandaloneSchedulerBackend.ACTOR_NAME)
     val command = "\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
-      runScript, masterUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)
+      runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)
     val environment = Environment.newBuilder()
     sc.executorEnvs.foreach { case (key, value) =>
       environment.addVariables(Environment.Variable.newBuilder()
@@ -184,7 +170,7 @@ private[spark] class CoarseMesosSchedulerBackend(
   }
 
   /** Helper function to pull out a resource from a Mesos Resources protobuf */
-  def getResource(res: JList[Resource], name: String): Double = {
+  private def getResource(res: JList[Resource], name: String): Double = {
     for (r <- res if r.getName == name) {
       return r.getScalar.getValue
     }
@@ -193,7 +179,7 @@ private[spark] class CoarseMesosSchedulerBackend(
   }
 
   /** Build a Mesos resource protobuf object */
-  def createResource(resourceName: String, quantity: Double): Protos.Resource = {
+  private def createResource(resourceName: String, quantity: Double): Protos.Resource = {
     Resource.newBuilder()
       .setName(resourceName)
       .setType(Value.Type.SCALAR)
@@ -202,7 +188,7 @@ private[spark] class CoarseMesosSchedulerBackend(
   }
 
   /** Check whether a Mesos task state represents a finished task */
-  def isFinished(state: MesosTaskState) = {
+  private def isFinished(state: MesosTaskState) = {
     state == MesosTaskState.TASK_FINISHED ||
       state == MesosTaskState.TASK_FAILED ||
       state == MesosTaskState.TASK_KILLED ||
@@ -253,7 +239,11 @@ private[spark] class CoarseMesosSchedulerBackend(
   override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) {
     logInfo("Mesos slave lost: " + slaveId.getValue)
     synchronized {
-      slaveIdsWithExecutors -= slaveId.getValue
+      if (slaveIdsWithExecutors.contains(slaveId.getValue)) {
+        // Note that the slave ID corresponds to the executor ID on that slave
+        slaveIdsWithExecutors -= slaveId.getValue
+        removeExecutor(slaveId.getValue, "Mesos slave lost")
+      }
     }
   }
 
diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
index 8c7a1dfbc0570b80af2517a7a572eef106c9badd..ca7fab4cc5ff95f318b33fc2b82e09f870d83f03 100644
--- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
@@ -24,21 +24,11 @@ private[spark] class MesosSchedulerBackend(
     scheduler: ClusterScheduler,
     sc: SparkContext,
     master: String,
-    frameworkName: String)
+    appName: String)
   extends SchedulerBackend
   with MScheduler
   with Logging {
 
-  // Memory used by each executor (in megabytes)
-  val EXECUTOR_MEMORY = {
-    if (System.getenv("SPARK_MEM") != null) {
-      Utils.memoryStringToMb(System.getenv("SPARK_MEM"))
-      // TODO: Might need to add some extra memory for the non-heap parts of the JVM
-    } else {
-      512
-    }
-  }
-
   // Lock used to wait for scheduler to be registered
   var isRegistered = false
   val registeredLock = new Object()
@@ -51,7 +41,7 @@ private[spark] class MesosSchedulerBackend(
   val taskIdToSlaveId = new HashMap[Long, String]
 
   // An ExecutorInfo for our tasks
-  var executorInfo: ExecutorInfo = null
+  var execArgs: Array[Byte] = null
 
   override def start() {
     synchronized {
@@ -59,7 +49,7 @@ private[spark] class MesosSchedulerBackend(
         setDaemon(true)
         override def run() {
           val scheduler = MesosSchedulerBackend.this
-          val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(frameworkName).build()
+          val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build()
           driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
           try {
             val ret = driver.run()
@@ -70,19 +60,14 @@ private[spark] class MesosSchedulerBackend(
         }
       }.start()
 
-      executorInfo = createExecutorInfo()
       waitForRegister()
     }
   }
 
-  def createExecutorInfo(): ExecutorInfo = {
-    val sparkHome = sc.getSparkHome() match {
-      case Some(path) =>
-        path
-      case None =>
-        throw new SparkException("Spark home is not set; set it through the spark.home system " +
-          "property, the SPARK_HOME environment variable or the SparkContext constructor")
-    }
+  def createExecutorInfo(execId: String): ExecutorInfo = {
+    val sparkHome = sc.getSparkHome().getOrElse(throw new SparkException(
+      "Spark home is not set; set it through the spark.home system " +
+      "property, the SPARK_HOME environment variable or the SparkContext constructor"))
     val execScript = new File(sparkHome, "spark-executor").getCanonicalPath
     val environment = Environment.newBuilder()
     sc.executorEnvs.foreach { case (key, value) =>
@@ -94,14 +79,14 @@ private[spark] class MesosSchedulerBackend(
     val memory = Resource.newBuilder()
       .setName("mem")
       .setType(Value.Type.SCALAR)
-      .setScalar(Value.Scalar.newBuilder().setValue(EXECUTOR_MEMORY).build())
+      .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build())
       .build()
     val command = CommandInfo.newBuilder()
       .setValue(execScript)
       .setEnvironment(environment)
       .build()
     ExecutorInfo.newBuilder()
-      .setExecutorId(ExecutorID.newBuilder().setValue("default").build())
+      .setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
       .setCommand(command)
       .setData(ByteString.copyFrom(createExecArg()))
       .addResources(memory)
@@ -113,17 +98,20 @@ private[spark] class MesosSchedulerBackend(
    * containing all the spark.* system properties in the form of (String, String) pairs.
    */
   private def createExecArg(): Array[Byte] = {
-    val props = new HashMap[String, String]
-    val iterator = System.getProperties.entrySet.iterator
-    while (iterator.hasNext) {
-      val entry = iterator.next
-      val (key, value) = (entry.getKey.toString, entry.getValue.toString)
-      if (key.startsWith("spark.")) {
-        props(key) = value
+    if (execArgs == null) {
+      val props = new HashMap[String, String]
+      val iterator = System.getProperties.entrySet.iterator
+      while (iterator.hasNext) {
+        val entry = iterator.next
+        val (key, value) = (entry.getKey.toString, entry.getValue.toString)
+        if (key.startsWith("spark.")) {
+          props(key) = value
+        }
       }
+      // Serialize the map as an array of (String, String) pairs
+      execArgs = Utils.serialize(props.toArray)
     }
-    // Serialize the map as an array of (String, String) pairs
-    return Utils.serialize(props.toArray)
+    return execArgs
   }
 
   override def offerRescinded(d: SchedulerDriver, o: OfferID) {}
@@ -163,7 +151,7 @@ private[spark] class MesosSchedulerBackend(
       def enoughMemory(o: Offer) = {
         val mem = getResource(o.getResourcesList, "mem")
         val slaveId = o.getSlaveId.getValue
-        mem >= EXECUTOR_MEMORY || slaveIdsWithExecutors.contains(slaveId)
+        mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId)
       }
 
       for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) {
@@ -220,7 +208,7 @@ private[spark] class MesosSchedulerBackend(
     return MesosTaskInfo.newBuilder()
       .setTaskId(taskId)
       .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build())
-      .setExecutor(executorInfo)
+      .setExecutor(createExecutorInfo(slaveId))
       .setName(task.name)
       .addResources(cpuResource)
       .setData(ByteString.copyFrom(task.serializedTask))
@@ -272,7 +260,7 @@ private[spark] class MesosSchedulerBackend(
     synchronized {
       slaveIdsWithExecutors -= slaveId.getValue
     }
-    scheduler.slaveLost(slaveId.getValue, reason)
+    scheduler.executorLost(slaveId.getValue, reason)
   }
 
   override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) {
diff --git a/core/src/main/scala/spark/serializer/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala
index 50b086125a55e2217f5877c54c708fae8f4e8a3b..aca86ab6f0f1d93390bab2eeb2a197d2ca03a856 100644
--- a/core/src/main/scala/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/spark/serializer/Serializer.scala
@@ -72,40 +72,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/storage/BlockFetchTracker.scala b/core/src/main/scala/spark/storage/BlockFetchTracker.scala
new file mode 100644
index 0000000000000000000000000000000000000000..993aece1f7f76b8550dd1b6815e98c33d60e1a0a
--- /dev/null
+++ b/core/src/main/scala/spark/storage/BlockFetchTracker.scala
@@ -0,0 +1,10 @@
+package spark.storage
+
+private[spark] trait BlockFetchTracker {
+    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/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala
index 7a1344668fbcad484458ed9e82b9fa12ef8a4723..8f737c5c6a8030fa31b995a852af82bac057cc9d 100644
--- a/core/src/main/scala/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/spark/storage/BlockManager.scala
@@ -16,7 +16,7 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
 
 import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
 
-import spark.{CacheTracker, Logging, SizeEstimator, SparkEnv, SparkException, Utils}
+import spark.{Logging, SizeEstimator, SparkEnv, SparkException, Utils}
 import spark.network._
 import spark.serializer.Serializer
 import spark.util.{ByteBufferInputStream, IdGenerator, MetadataCleaner, TimeStampedHashMap}
@@ -30,6 +30,7 @@ extends Exception(message)
 
 private[spark]
 class BlockManager(
+    executorId: String,
     actorSystem: ActorSystem,
     val master: BlockManagerMaster,
     val serializer: Serializer,
@@ -39,21 +40,36 @@ class BlockManager(
   class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
     var pending: Boolean = true
     var size: Long = -1L
+    var failed: Boolean = false
 
-    /** Wait for this BlockInfo to be marked as ready (i.e. block is finished writing) */
-    def waitForReady() {
+    /**
+     * 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) {
         synchronized {
           while (pending) this.wait()
         }
       }
+      !failed
     }
 
     /** Mark this BlockInfo as ready (i.e. block is finished writing) */
     def markReady(sizeInBytes: Long) {
-      pending = false
-      size = sizeInBytes
       synchronized {
+        pending = false
+        failed = false
+        size = sizeInBytes
+        this.notifyAll()
+      }
+    }
+
+    /** Mark this BlockInfo as ready but failed */
+    def markFailure() {
+      synchronized {
+        failed = true
+        pending = false
         this.notifyAll()
       }
     }
@@ -68,11 +84,8 @@ class BlockManager(
   val connectionManager = new ConnectionManager(0)
   implicit val futureExecContext = connectionManager.futureExecContext
 
-  val connectionManagerId = connectionManager.id
-  val blockManagerId = new BlockManagerId(connectionManagerId.host, connectionManagerId.port)
-
-  // TODO: This will be removed after cacheTracker is removed from the code base.
-  var cacheTracker: CacheTracker = null
+  val blockManagerId = BlockManagerId(
+    executorId, connectionManager.id.host, connectionManager.id.port)
 
   // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory
   // for receiving shuffle outputs)
@@ -90,10 +103,13 @@ class BlockManager(
 
   val host = System.getProperty("spark.hostname", Utils.localHostName())
 
-  val slaveActor = master.actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
+  val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
     name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next)
 
-  @volatile private var shuttingDown = false
+  // Pending reregistration action being executed asynchronously or null if none
+  // is pending. Accesses should synchronize on asyncReregisterLock.
+  var asyncReregisterTask: Future[Unit] = null
+  val asyncReregisterLock = new Object
 
   private def heartBeat() {
     if (!master.sendHeartBeat(blockManagerId)) {
@@ -109,8 +125,9 @@ class BlockManager(
   /**
    * Construct a BlockManager with a memory limit set based on system properties.
    */
-  def this(actorSystem: ActorSystem, master: BlockManagerMaster, serializer: Serializer) = {
-    this(actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties)
+  def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster,
+           serializer: Serializer) = {
+    this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties)
   }
 
   /**
@@ -150,6 +167,8 @@ class BlockManager(
   /**
    * Reregister with the master and report all blocks to it. This will be called by the heart beat
    * thread if our heartbeat to the block amnager indicates that we were not registered.
+   *
+   * Note that this method must be called without any BlockInfo locks held.
    */
   def reregister() {
     // TODO: We might need to rate limit reregistering.
@@ -158,6 +177,32 @@ class BlockManager(
     reportAllBlocks()
   }
 
+  /**
+   * Reregister with the master sometime soon.
+   */
+  def asyncReregister() {
+    asyncReregisterLock.synchronized {
+      if (asyncReregisterTask == null) {
+        asyncReregisterTask = Future[Unit] {
+          reregister()
+          asyncReregisterLock.synchronized {
+            asyncReregisterTask = null
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * For testing. Wait for any pending asynchronous reregistration; otherwise, do nothing.
+   */
+  def waitForAsyncReregister() {
+    val task = asyncReregisterTask
+    if (task != null) {
+      Await.ready(task, Duration.Inf)
+    }
+  }
+
   /**
    * Get storage level of local block. If no info exists for the block, then returns null.
    */
@@ -173,7 +218,7 @@ class BlockManager(
     if (needReregister) {
       logInfo("Got told to reregister updating block " + blockId)
       // Reregistering will report our new block for free.
-      reregister()
+      asyncReregister()
     }
     logDebug("Told master about block " + blockId)
   }
@@ -191,7 +236,7 @@ class BlockManager(
         case level =>
           val inMem = level.useMemory && memoryStore.contains(blockId)
           val onDisk = level.useDisk && diskStore.contains(blockId)
-          val storageLevel = new StorageLevel(onDisk, inMem, level.deserialized, level.replication)
+          val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication)
           val memSize = if (inMem) memoryStore.getSize(blockId) else 0L
           val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L
           (storageLevel, memSize, diskSize, info.tellMaster)
@@ -213,7 +258,7 @@ class BlockManager(
     val startTimeMs = System.currentTimeMillis
     var managers = master.getLocations(blockId)
     val locations = managers.map(_.ip)
-    logDebug("Get block locations in " + Utils.getUsedTimeMs(startTimeMs))
+    logDebug("Got block locations in " + Utils.getUsedTimeMs(startTimeMs))
     return locations
   }
 
@@ -223,7 +268,7 @@ class BlockManager(
   def getLocations(blockIds: Array[String]): Array[Seq[String]] = {
     val startTimeMs = System.currentTimeMillis
     val locations = master.getLocations(blockIds).map(_.map(_.ip).toSeq).toArray
-    logDebug("Get multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
+    logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
     return locations
   }
 
@@ -247,7 +292,14 @@ class BlockManager(
     val info = blockInfo.get(blockId).orNull
     if (info != null) {
       info.synchronized {
-        info.waitForReady() // In case the block is still being put() by another thread
+
+        // In the another thread is writing the block, wait for it to become ready.
+        if (!info.waitForReady()) {
+          // If we get here, the block write failed.
+          logWarning("Block " + blockId + " was marked as failure.")
+          return None
+        }
+
         val level = info.level
         logDebug("Level for block " + blockId + " is " + level)
 
@@ -332,7 +384,14 @@ class BlockManager(
     val info = blockInfo.get(blockId).orNull
     if (info != null) {
       info.synchronized {
-        info.waitForReady() // In case the block is still being put() by another thread
+
+        // In the another thread is writing the block, wait for it to become ready.
+        if (!info.waitForReady()) {
+          // If we get here, the block write failed.
+          logWarning("Block " + blockId + " was marked as failure.")
+          return None
+        }
+
         val level = info.level
         logDebug("Level for block " + blockId + " is " + level)
 
@@ -393,12 +452,11 @@ class BlockManager(
       val data = BlockManagerWorker.syncGetBlock(
           GetBlock(blockId), ConnectionManagerId(loc.ip, loc.port))
       if (data != null) {
-        logDebug("Data is not null: " + data)
         return Some(dataDeserialize(blockId, data))
       }
-      logDebug("Data is null")
+      logDebug("The value of block " + blockId + " is null")
     }
-    logDebug("Data not found")
+    logDebug("Block " + blockId + " not found")
     return None
   }
 
@@ -416,152 +474,8 @@ class BlockManager(
    * so that we can control the maxMegabytesInFlight for the fetch.
    */
   def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])])
-      : Iterator[(String, Option[Iterator[Any]])] = {
-
-    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
-    }
-
-    // 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]
-
-    // Current bytes in flight from our requests
-    var bytesInFlight = 0L
-
-    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 future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage)
-      future.onSuccess {
-        case Some(message) => {
-          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)))
-            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))
-          }
-        }
-      }
-    }
-
-    // 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)
-        }
-      }
-    }
-    // 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")
-
-    // Return an iterator that will read fetched blocks off the queue as they arrive.
-    return new Iterator[(String, Option[Iterator[Any]])] {
-      var resultsGotten = 0
-
-      def hasNext: Boolean = resultsGotten < totalBlocks
-
-      def next(): (String, Option[Iterator[Any]]) = {
-        resultsGotten += 1
-        val result = results.take()
-        bytesInFlight -= result.size
-        if (!fetchRequests.isEmpty &&
-            (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
-          sendRequest(fetchRequests.dequeue())
-        }
-        (result.blockId, if (result.failed) None else Some(result.deserialize()))
-      }
-    }
+      : BlockFetcherIterator = {
+    return new BlockFetcherIterator(this, blocksByAddress)
   }
 
   def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
@@ -588,9 +502,8 @@ class BlockManager(
     }
 
     val oldBlock = blockInfo.get(blockId).orNull
-    if (oldBlock != null) {
+    if (oldBlock != null && oldBlock.waitForReady()) {
       logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
-      oldBlock.waitForReady()
       return oldBlock.size
     }
 
@@ -615,40 +528,56 @@ class BlockManager(
     var size = 0L
 
     myInfo.synchronized {
-      logDebug("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
+      logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
         + " to get into synchronized block")
 
-      if (level.useMemory) {
-        // Save it just to memory first, even if it also has useDisk set to true; we will later
-        // drop it to disk if the memory store can't hold it.
-        val res = memoryStore.putValues(blockId, values, level, true)
-        size = res.size
-        res.data match {
-          case Right(newBytes) => bytesAfterPut = newBytes
-          case Left(newIterator) => valuesAfterPut = newIterator
-        }
-      } else {
-        // Save directly to disk.
-        val askForBytes = level.replication > 1 // Don't get back the bytes unless we replicate them
-        val res = diskStore.putValues(blockId, values, level, askForBytes)
-        size = res.size
-        res.data match {
-          case Right(newBytes) => bytesAfterPut = newBytes
-          case _ =>
+      try {
+        if (level.useMemory) {
+          // Save it just to memory first, even if it also has useDisk set to true; we will later
+          // drop it to disk if the memory store can't hold it.
+          val res = memoryStore.putValues(blockId, values, level, true)
+          size = res.size
+          res.data match {
+            case Right(newBytes) => bytesAfterPut = newBytes
+            case Left(newIterator) => valuesAfterPut = newIterator
+          }
+        } else {
+          // Save directly to disk.
+          // Don't get back the bytes unless we replicate them.
+          val askForBytes = level.replication > 1
+          val res = diskStore.putValues(blockId, values, level, askForBytes)
+          size = res.size
+          res.data match {
+            case Right(newBytes) => bytesAfterPut = newBytes
+            case _ =>
+          }
         }
-      }
 
-      // Now that the block is in either the memory or disk store, let other threads read it,
-      // and tell the master about it.
-      myInfo.markReady(size)
-      if (tellMaster) {
-        reportBlockStatus(blockId, myInfo)
+        // Now that the block is in either the memory or disk store, let other threads read it,
+        // and tell the master about it.
+        myInfo.markReady(size)
+        if (tellMaster) {
+          reportBlockStatus(blockId, myInfo)
+        }
+      } catch {
+        // 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 => {
+          // 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
+        }
       }
     }
     logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs))
 
+
     // Replicate block if required
     if (level.replication > 1) {
+      val remoteStartTime = System.currentTimeMillis
       // Serialize the block if not already done
       if (bytesAfterPut == null) {
         if (valuesAfterPut == null) {
@@ -658,16 +587,10 @@ class BlockManager(
         bytesAfterPut = dataSerialize(blockId, valuesAfterPut)
       }
       replicate(blockId, bytesAfterPut, level)
+      logDebug("Put block " + blockId + " remotely took " + Utils.getUsedTimeMs(remoteStartTime))
     }
-
     BlockManager.dispose(bytesAfterPut)
 
-    // TODO: This code will be removed when CacheTracker is gone.
-    if (blockId.startsWith("rdd")) {
-      notifyCacheTracker(blockId)
-    }
-    logDebug("Put block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs))
-
     return size
   }
 
@@ -716,33 +639,38 @@ class BlockManager(
       logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
         + " to get into synchronized block")
 
-      if (level.useMemory) {
-        // Store it only in memory at first, even if useDisk is also set to true
-        bytes.rewind()
-        memoryStore.putBytes(blockId, bytes, level)
-      } else {
-        bytes.rewind()
-        diskStore.putBytes(blockId, bytes, level)
-      }
+      try {
+        if (level.useMemory) {
+          // Store it only in memory at first, even if useDisk is also set to true
+          bytes.rewind()
+          memoryStore.putBytes(blockId, bytes, level)
+        } else {
+          bytes.rewind()
+          diskStore.putBytes(blockId, bytes, level)
+        }
 
-      // Now that the block is in either the memory or disk store, let other threads read it,
-      // and tell the master about it.
-      myInfo.markReady(bytes.limit)
-      if (tellMaster) {
-        reportBlockStatus(blockId, myInfo)
+        // Now that the block is in either the memory or disk store, let other threads read it,
+        // and tell the master about it.
+        myInfo.markReady(bytes.limit)
+        if (tellMaster) {
+          reportBlockStatus(blockId, myInfo)
+        }
+      } catch {
+        // 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 => {
+          // 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
+        }
       }
     }
 
-    // TODO: This code will be removed when CacheTracker is gone.
-    if (blockId.startsWith("rdd")) {
-      notifyCacheTracker(blockId)
-    }
-
     // If replication had started, then wait for it to finish
     if (level.replication > 1) {
-      if (replicationFuture == null) {
-        throw new Exception("Unexpected")
-      }
       Await.ready(replicationFuture, Duration.Inf)
     }
 
@@ -760,8 +688,7 @@ class BlockManager(
    */
   var cachedPeers: Seq[BlockManagerId] = null
   private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) {
-    val tLevel: StorageLevel =
-      new StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
+    val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
     if (cachedPeers == null) {
       cachedPeers = master.getPeers(blockManagerId, level.replication - 1)
     }
@@ -780,16 +707,6 @@ class BlockManager(
     }
   }
 
-  // TODO: This code will be removed when CacheTracker is gone.
-  private def notifyCacheTracker(key: String) {
-    if (cacheTracker != null) {
-      val rddInfo = key.split("_")
-      val rddId: Int = rddInfo(1).toInt
-      val partition: Int = rddInfo(2).toInt
-      cacheTracker.notifyFromBlockManager(spark.AddedToCache(rddId, partition, host))
-    }
-  }
-
   /**
    * Read a block consisting of a single object.
    */
@@ -936,10 +853,11 @@ class BlockManager(
       heartBeatTask.cancel()
     }
     connectionManager.stop()
-    master.actorSystem.stop(slaveActor)
+    actorSystem.stop(slaveActor)
     blockInfo.clear()
     memoryStore.clear()
     diskStore.clear()
+    metadataCleaner.cancel()
     logInfo("BlockManager stopped")
   }
 }
@@ -968,10 +886,183 @@ object BlockManager extends Logging {
    */
   def dispose(buffer: ByteBuffer) {
     if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) {
-      logDebug("Unmapping " + buffer)
+      logTrace("Unmapping " + buffer)
       if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) {
         buffer.asInstanceOf[DirectBuffer].cleaner().clean()
       }
     }
   }
 }
+
+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 _fetchWaitTime = 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
+  }
+
+  // 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]
+
+  // Current bytes in flight from our requests
+  var bytesInFlight = 0L
+
+  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))
+        }
+      }
+      case None => {
+        logError("Could not get block(s) from " + cmId)
+        for ((blockId, size) <- req.blocks) {
+          results.put(new FetchResult(blockId, -1, null))
+        }
+      }
+    }
+  }
+
+  // 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)
+      }
+    }
+  }
+  // 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()
+    _fetchWaitTime += (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()))
+  }
+
+
+  //methods to profile the block fetching
+  def numLocalBlocks = localBlockIds.size
+  def numRemoteBlocks = remoteBlockIds.size
+
+  def remoteFetchTime = _remoteFetchTime
+  def fetchWaitTime = _fetchWaitTime
+
+  def remoteBytesRead = _remoteBytesRead
+
+}
diff --git a/core/src/main/scala/spark/storage/BlockManagerId.scala b/core/src/main/scala/spark/storage/BlockManagerId.scala
index 488679f0496b42d4cbb59ba6205583a4992584b0..f2f1e77d41a65cb810fd58904626dccb7b5caaff 100644
--- a/core/src/main/scala/spark/storage/BlockManagerId.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerId.scala
@@ -3,38 +3,67 @@ package spark.storage
 import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
 import java.util.concurrent.ConcurrentHashMap
 
+/**
+ * This class represent an unique identifier for a BlockManager.
+ * The first 2 constructors of this class is made private to ensure that
+ * BlockManagerId objects can be created only using the factory method in
+ * [[spark.storage.BlockManager$]]. This allows de-duplication of ID objects.
+ * Also, constructor parameters are private to ensure that parameters cannot
+ * be modified from outside this class.
+ */
+private[spark] class BlockManagerId private (
+    private var executorId_ : String,
+    private var ip_ : String,
+    private var port_ : Int
+  ) extends Externalizable {
 
-private[spark] class BlockManagerId(var ip: String, var port: Int) extends Externalizable {
-  def this() = this(null, 0)  // For deserialization only
+  private def this() = this(null, null, 0)  // For deserialization only
 
-  def this(in: ObjectInput) = this(in.readUTF(), in.readInt())
+  def executorId: String = executorId_
+
+  def ip: String = ip_
+
+  def port: Int = port_
 
   override def writeExternal(out: ObjectOutput) {
-    out.writeUTF(ip)
-    out.writeInt(port)
+    out.writeUTF(executorId_)
+    out.writeUTF(ip_)
+    out.writeInt(port_)
   }
 
   override def readExternal(in: ObjectInput) {
-    ip = in.readUTF()
-    port = in.readInt()
+    executorId_ = in.readUTF()
+    ip_ = in.readUTF()
+    port_ = in.readInt()
   }
 
   @throws(classOf[IOException])
   private def readResolve(): Object = BlockManagerId.getCachedBlockManagerId(this)
 
-  override def toString = "BlockManagerId(" + ip + ", " + port + ")"
+  override def toString = "BlockManagerId(%s, %s, %d)".format(executorId, ip, port)
 
-  override def hashCode = ip.hashCode * 41 + port
+  override def hashCode: Int = (executorId.hashCode * 41 + ip.hashCode) * 41 + port
 
   override def equals(that: Any) = that match {
-    case id: BlockManagerId => port == id.port && ip == id.ip
-    case _ => false
+    case id: BlockManagerId =>
+      executorId == id.executorId && port == id.port && ip == id.ip
+    case _ =>
+      false
   }
 }
 
 
 private[spark] object BlockManagerId {
 
+  def apply(execId: String, ip: String, port: Int) =
+    getCachedBlockManagerId(new BlockManagerId(execId, ip, port))
+
+  def apply(in: ObjectInput) = {
+    val obj = new BlockManagerId()
+    obj.readExternal(in)
+    getCachedBlockManagerId(obj)
+  }
+
   val blockManagerIdCache = new ConcurrentHashMap[BlockManagerId, BlockManagerId]()
 
   def getCachedBlockManagerId(id: BlockManagerId): BlockManagerId = {
diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
index 46e1860d09fff10868c8a4e8c7093f1b2cd12c91..4e55936d2873477cd8d9143c6b5bd7d1d9a5b85f 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
@@ -1,6 +1,10 @@
 package spark.storage
 
-import scala.collection.mutable.ArrayBuffer
+import java.io._
+import java.util.{HashMap => JHashMap}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.util.Random
 
 import akka.actor.{Actor, ActorRef, ActorSystem, Props}
@@ -10,54 +14,33 @@ import scala.concurrent.duration._
 
 import spark.{Logging, SparkException, Utils}
 
+private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
 
-private[spark] class BlockManagerMaster(
-    val actorSystem: ActorSystem,
-    isMaster: Boolean,
-    isLocal: Boolean,
-    masterIp: String,
-    masterPort: Int)
-  extends Logging {
-
-  val AKKA_RETRY_ATTEMPS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
+  val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
   val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt
 
-  val MASTER_AKKA_ACTOR_NAME = "BlockMasterManager"
-  val SLAVE_AKKA_ACTOR_NAME = "BlockSlaveManager"
-  val DEFAULT_MANAGER_IP: String = Utils.localHostName()
+  val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
 
   val timeout = 10.seconds
-  var masterActor: ActorRef = {
-    if (isMaster) {
-      val masterActor = actorSystem.actorOf(Props(new BlockManagerMasterActor(isLocal)),
-        name = MASTER_AKKA_ACTOR_NAME)
-      logInfo("Registered BlockManagerMaster Actor")
-      masterActor
-    } else {
-      val url = "akka://spark@%s:%s/user/%s".format(masterIp, masterPort, MASTER_AKKA_ACTOR_NAME)
-      logInfo("Connecting to BlockManagerMaster: " + url)
-      actorSystem.actorFor(url)
-    }
-  }
 
-  /** Remove a dead host from the master actor. This is only called on the master side. */
-  def notifyADeadHost(host: String) {
-    tell(RemoveHost(host))
-    logInfo("Removed " + host + " successfully in notifyADeadHost")
+  /** Remove a dead executor from the driver actor. This is only called on the driver side. */
+  def removeExecutor(execId: String) {
+    tell(RemoveExecutor(execId))
+    logInfo("Removed " + execId + " successfully in removeExecutor")
   }
 
   /**
-   * Send the master actor a heart beat from the slave. Returns true if everything works out,
-   * false if the master does not know about the given block manager, which means the block
+   * Send the driver actor a heart beat from the slave. Returns true if everything works out,
+   * false if the driver does not know about the given block manager, which means the block
    * manager should re-register.
    */
   def sendHeartBeat(blockManagerId: BlockManagerId): Boolean = {
-    askMasterWithRetry[Boolean](HeartBeat(blockManagerId))
+    askDriverWithReply[Boolean](HeartBeat(blockManagerId))
   }
 
-  /** Register the BlockManager's id with the master. */
+  /** Register the BlockManager's id with the driver. */
   def registerBlockManager(
-    blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
+      blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
     logInfo("Trying to register BlockManager")
     tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor))
     logInfo("Registered BlockManager")
@@ -69,25 +52,25 @@ private[spark] class BlockManagerMaster(
       storageLevel: StorageLevel,
       memSize: Long,
       diskSize: Long): Boolean = {
-    val res = askMasterWithRetry[Boolean](
+    val res = askDriverWithReply[Boolean](
       UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize))
     logInfo("Updated info of block " + blockId)
     res
   }
 
-  /** Get locations of the blockId from the master */
+  /** Get locations of the blockId from the driver */
   def getLocations(blockId: String): Seq[BlockManagerId] = {
-    askMasterWithRetry[Seq[BlockManagerId]](GetLocations(blockId))
+    askDriverWithReply[Seq[BlockManagerId]](GetLocations(blockId))
   }
 
-  /** Get locations of multiple blockIds from the master */
+  /** Get locations of multiple blockIds from the driver */
   def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
-    askMasterWithRetry[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds))
+    askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds))
   }
 
-  /** Get ids of other nodes in the cluster from the master */
+  /** Get ids of other nodes in the cluster from the driver */
   def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = {
-    val result = askMasterWithRetry[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers))
+    val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers))
     if (result.length != numPeers) {
       throw new SparkException(
         "Error getting peers, only got " + result.size + " instead of " + numPeers)
@@ -97,10 +80,10 @@ private[spark] class BlockManagerMaster(
 
   /**
    * Remove a block from the slaves that have it. This can only be used to remove
-   * blocks that the master knows about.
+   * blocks that the driver knows about.
    */
   def removeBlock(blockId: String) {
-    askMasterWithRetry(RemoveBlock(blockId))
+    askDriverWithReply(RemoveBlock(blockId))
   }
 
   /**
@@ -110,41 +93,45 @@ private[spark] class BlockManagerMaster(
    * amount of remaining memory.
    */
   def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = {
-    askMasterWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus)
+    askDriverWithReply[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus)
+  }
+
+  def getStorageStatus: Array[StorageStatus] = {
+    askDriverWithReply[ArrayBuffer[StorageStatus]](GetStorageStatus).toArray
   }
 
-  /** Stop the master actor, called only on the Spark master node */
+  /** Stop the driver actor, called only on the Spark driver node */
   def stop() {
-    if (masterActor != null) {
+    if (driverActor != null) {
       tell(StopBlockManagerMaster)
-      masterActor = null
+      driverActor = null
       logInfo("BlockManagerMaster stopped")
     }
   }
 
   /** Send a one-way message to the master actor, to which we expect it to reply with true. */
   private def tell(message: Any) {
-    if (!askMasterWithRetry[Boolean](message)) {
+    if (!askDriverWithReply[Boolean](message)) {
       throw new SparkException("BlockManagerMasterActor returned false, expected true.")
     }
   }
 
   /**
-   * Send a message to the master actor and get its result within a default timeout, or
+   * Send a message to the driver actor and get its result within a default timeout, or
    * throw a SparkException if this fails.
    */
-  private def askMasterWithRetry[T](message: Any): T = {
+  private def askDriverWithReply[T](message: Any): T = {
     // TODO: Consider removing multiple attempts
-    if (masterActor == null) {
-      throw new SparkException("Error sending message to BlockManager as masterActor is null " +
+    if (driverActor == null) {
+      throw new SparkException("Error sending message to BlockManager as driverActor is null " +
         "[message = " + message + "]")
     }
     var attempts = 0
     var lastException: Exception = null
-    while (attempts < AKKA_RETRY_ATTEMPS) {
+    while (attempts < AKKA_RETRY_ATTEMPTS) {
       attempts += 1
       try {
-        val future = masterActor.ask(message)(timeout)
+        val future = driverActor.ask(message)(timeout)
         val result = Await.result(future, timeout)
         if (result == null) {
           throw new Exception("BlockManagerMaster returned null")
diff --git a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
index 9bf9161c312552c931326322fe4ca9cdf9a258b9..2d39e2c15c5dab47259f54c8ee50617c0d9f34f7 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMasterActor.scala
@@ -22,9 +22,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
   private val blockManagerInfo =
     new HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo]
 
-  // Mapping from host name to block manager id. We allow multiple block managers
-  // on the same host name (ip).
-  private val blockManagerIdByHost = new HashMap[String, ArrayBuffer[BlockManagerId]]
+  // Mapping from executor ID to block manager ID.
+  private val blockManagerIdByExecutor = new HashMap[String, BlockManagerId]
 
   // Mapping from block id to the set of block managers that have the block.
   private val blockLocations = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]]
@@ -68,11 +67,14 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
     case GetMemoryStatus =>
       getMemoryStatus
 
+    case GetStorageStatus =>
+      getStorageStatus
+
     case RemoveBlock(blockId) =>
       removeBlock(blockId)
 
-    case RemoveHost(host) =>
-      removeHost(host)
+    case RemoveExecutor(execId) =>
+      removeExecutor(execId)
       sender ! true
 
     case StopBlockManagerMaster =>
@@ -96,16 +98,12 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
   def removeBlockManager(blockManagerId: BlockManagerId) {
     val info = blockManagerInfo(blockManagerId)
 
-    // Remove the block manager from blockManagerIdByHost. If the list of block
-    // managers belonging to the IP is empty, remove the entry from the hash map.
-    blockManagerIdByHost.get(blockManagerId.ip).foreach { managers: ArrayBuffer[BlockManagerId] =>
-      managers -= blockManagerId
-      if (managers.size == 0) blockManagerIdByHost.remove(blockManagerId.ip)
-    }
+    // Remove the block manager from blockManagerIdByExecutor.
+    blockManagerIdByExecutor -= blockManagerId.executorId
 
     // Remove it from blockManagerInfo and remove all the blocks.
     blockManagerInfo.remove(blockManagerId)
-    var iterator = info.blocks.keySet.iterator
+    val iterator = info.blocks.keySet.iterator
     while (iterator.hasNext) {
       val blockId = iterator.next
       val locations = blockLocations.get(blockId)._2
@@ -117,30 +115,29 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
   }
 
   def expireDeadHosts() {
-    logDebug("Checking for hosts with no recent heart beats in BlockManagerMaster.")
+    logTrace("Checking for hosts with no recent heart beats in BlockManagerMaster.")
     val now = System.currentTimeMillis()
     val minSeenTime = now - slaveTimeout
     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
       }
     }
     toRemove.foreach(removeBlockManager)
   }
 
-  def removeHost(host: String) {
-    logInfo("Trying to remove the host: " + host + " from BlockManagerMaster.")
-    logInfo("Previous hosts: " + blockManagerInfo.keySet.toSeq)
-    blockManagerIdByHost.get(host).foreach(_.foreach(removeBlockManager))
-    logInfo("Current hosts: " + blockManagerInfo.keySet.toSeq)
+  def removeExecutor(execId: String) {
+    logInfo("Trying to remove executor " + execId + " from BlockManagerMaster.")
+    blockManagerIdByExecutor.get(execId).foreach(removeBlockManager)
     sender ! true
   }
 
   def heartBeat(blockManagerId: BlockManagerId) {
     if (!blockManagerInfo.contains(blockManagerId)) {
-      if (blockManagerId.ip == Utils.localHostName() && !isLocal) {
+      if (blockManagerId.executorId == "<driver>" && !isLocal) {
         sender ! true
       } else {
         sender ! false
@@ -177,24 +174,28 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
     sender ! res
   }
 
-  private def register(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
-    val startTimeMs = System.currentTimeMillis()
-    val tmp = " " + blockManagerId + " "
+  private def getStorageStatus() {
+    val res = blockManagerInfo.map { case(blockManagerId, info) =>
+      import collection.JavaConverters._
+      StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap)
+    }
+    sender ! res
+  }
 
-    if (blockManagerId.ip == Utils.localHostName() && !isLocal) {
-      logInfo("Got Register Msg from master node, don't register it")
-    } else {
-      blockManagerIdByHost.get(blockManagerId.ip) match {
-        case Some(managers) =>
-          // A block manager of the same host name already exists.
-          logInfo("Got another registration for host " + blockManagerId)
-          managers += blockManagerId
+  private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) {
+    if (id.executorId == "<driver>" && !isLocal) {
+      // Got a register message from the master node; don't register it
+    } else if (!blockManagerInfo.contains(id)) {
+      blockManagerIdByExecutor.get(id.executorId) match {
+        case Some(manager) =>
+          // A block manager of the same host name already exists
+          logError("Got two different block manager registrations on " + id.executorId)
+          System.exit(1)
         case None =>
-          blockManagerIdByHost += (blockManagerId.ip -> ArrayBuffer(blockManagerId))
+          blockManagerIdByExecutor(id.executorId) = id
       }
-
-      blockManagerInfo += (blockManagerId -> new BlockManagerMasterActor.BlockManagerInfo(
-        blockManagerId, System.currentTimeMillis(), maxMemSize, slaveActor))
+      blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo(
+        id, System.currentTimeMillis(), maxMemSize, slaveActor)
     }
     sender ! true
   }
@@ -206,11 +207,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
       memSize: Long,
       diskSize: Long) {
 
-    val startTimeMs = System.currentTimeMillis()
-    val tmp = " " + blockManagerId + " " + blockId + " "
-
     if (!blockManagerInfo.contains(blockManagerId)) {
-      if (blockManagerId.ip == Utils.localHostName() && !isLocal) {
+      if (blockManagerId.executorId == "<driver>" && !isLocal) {
         // We intentionally do not register the master (except in local mode),
         // so we should not indicate failure.
         sender ! true
@@ -342,8 +340,8 @@ object BlockManagerMasterActor {
       _lastSeenMs = System.currentTimeMillis()
     }
 
-    def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, diskSize: Long)
-      : Unit = synchronized {
+    def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long,
+                        diskSize: Long) {
 
       updateLastSeenMs()
 
diff --git a/core/src/main/scala/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/spark/storage/BlockManagerMessages.scala
index d73a9b790f4793596ed9fce418af83fc62a7edf5..cff48d9909a161047a11cffa5056aee6ecefd751 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMessages.scala
@@ -49,18 +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 = new BlockManagerId()
-    blockManagerId.readExternal(in)
+    blockManagerId = BlockManagerId(in)
     blockId = in.readUTF()
-    storageLevel = new StorageLevel()
-    storageLevel.readExternal(in)
-    memSize = in.readInt()
-    diskSize = in.readInt()
+    storageLevel = StorageLevel(in)
+    memSize = in.readLong()
+    diskSize = in.readLong()
   }
 }
 
@@ -90,7 +88,7 @@ private[spark]
 case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster
 
 private[spark]
-case class RemoveHost(host: String) extends ToBlockManagerMaster
+case class RemoveExecutor(execId: String) extends ToBlockManagerMaster
 
 private[spark]
 case object StopBlockManagerMaster extends ToBlockManagerMaster
@@ -100,3 +98,6 @@ case object GetMemoryStatus extends ToBlockManagerMaster
 
 private[spark]
 case object ExpireDeadHosts extends ToBlockManagerMaster
+
+private[spark]
+case object GetStorageStatus extends ToBlockManagerMaster
diff --git a/core/src/main/scala/spark/storage/BlockManagerUI.scala b/core/src/main/scala/spark/storage/BlockManagerUI.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9e6721ec17169a8ca33753393329049467b105dd
--- /dev/null
+++ b/core/src/main/scala/spark/storage/BlockManagerUI.scala
@@ -0,0 +1,76 @@
+package spark.storage
+
+import akka.actor.{ActorRef, ActorSystem}
+import akka.util.Timeout
+import akka.util.duration._
+import cc.spray.typeconversion.TwirlSupport._
+import cc.spray.Directives
+import spark.{Logging, SparkContext}
+import spark.util.AkkaUtils
+import spark.Utils
+
+
+/**
+ * Web UI server for the BlockManager inside each SparkContext.
+ */
+private[spark]
+class BlockManagerUI(val actorSystem: ActorSystem, blockManagerMaster: ActorRef, sc: SparkContext)
+  extends Directives with Logging {
+
+  val STATIC_RESOURCE_DIR = "spark/deploy/static"
+
+  implicit val timeout = Timeout(10 seconds)
+
+  /** 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))
+    } catch {
+      case e: Exception =>
+        logError("Failed to create BlockManager WebUI", e)
+        System.exit(1)
+    }
+  }
+
+  val handler = {
+    get {
+      path("") {
+        completeWith {
+          // Request the current storage status from the Master
+          val storageStatusList = sc.getExecutorStorageStatus
+          // Calculate macro-level statistics
+          val maxMem = storageStatusList.map(_.maxMem).reduce(_+_)
+          val remainingMem = storageStatusList.map(_.memRemaining).reduce(_+_)
+          val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize))
+            .reduceOption(_+_).getOrElse(0L)
+          val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc)
+          spark.storage.html.index.
+            render(maxMem, remainingMem, diskSpaceUsed, rdds, storageStatusList)
+        }
+      } ~
+      path("rdd") {
+        parameter("id") { id =>
+          completeWith {
+            val prefix = "rdd_" + id.toString
+            val storageStatusList = sc.getExecutorStorageStatus
+            val filteredStorageStatusList = StorageUtils.
+              filterStorageStatusByPrefix(storageStatusList, prefix)
+            val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head
+            spark.storage.html.rdd.render(rddInfo, filteredStorageStatusList)
+          }
+        }
+      } ~
+      pathPrefix("static") {
+        getFromResourceDirectory(STATIC_RESOURCE_DIR)
+      }
+    }
+  }
+}
diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala
index 3f234df654c34f44938947b8adac1200c1baa24f..30d7500e01e68dbdfbcad22232a1b8011f0daa78 100644
--- a/core/src/main/scala/spark/storage/BlockMessage.scala
+++ b/core/src/main/scala/spark/storage/BlockMessage.scala
@@ -64,7 +64,7 @@ private[spark] class BlockMessage() {
 
       val booleanInt = buffer.getInt()
       val replication = buffer.getInt()
-      level = new StorageLevel(booleanInt, replication)
+      level = StorageLevel(booleanInt, replication)
       
       val dataLength = buffer.getInt()
       data = ByteBuffer.allocate(dataLength)
diff --git a/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f6c28dce52ad554bdc06d20c1c0a2ad53d990018
--- /dev/null
+++ b/core/src/main/scala/spark/storage/DelegateBlockFetchTracker.scala
@@ -0,0 +1,12 @@
+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 fetchWaitTime = delegate.fetchWaitTime
+  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 7e5b820cbbdc6ca145c2eb7c6787bd2c137c80d0..ddbf8821ad15aa172cb248ef3aed45edb3e38f33 100644
--- a/core/src/main/scala/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/spark/storage/DiskStore.scala
@@ -178,7 +178,11 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
     Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") {
       override def run() {
         logDebug("Shutdown hook called")
-        localDirs.foreach(localDir => Utils.deleteRecursively(localDir))
+        try {
+          localDirs.foreach(localDir => Utils.deleteRecursively(localDir))
+        } catch {
+          case t: Throwable => logError("Exception while deleting local spark dirs", t)
+        }
       }
     })
   }
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala
index 00e32f753c1604507f7fc2b05eae0066729defa3..949588476c20150b1dd5c73f4303dbf85d2ad518 100644
--- a/core/src/main/scala/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/spark/storage/MemoryStore.scala
@@ -17,7 +17,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
 
   private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true)
   private var currentMemory = 0L
-
   // Object used to ensure that only one thread is putting blocks and if necessary, dropping
   // blocks from the memory store.
   private val putLock = new Object()
@@ -33,8 +32,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
   }
 
   override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
+    bytes.rewind()
     if (level.deserialized) {
-      bytes.rewind()
       val values = blockManager.dataDeserialize(blockId, bytes)
       val elements = new ArrayBuffer[Any]
       elements ++= values
@@ -59,7 +58,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
     } else {
       val bytes = blockManager.dataSerialize(blockId, values.iterator)
       tryToPut(blockId, bytes, bytes.limit, false)
-      PutResult(bytes.limit(), Right(bytes))
+      PutResult(bytes.limit(), Right(bytes.duplicate()))
     }
   }
 
diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala
index e3544e5aae28ca5d41ba524ed980e5af98c4f3d7..3b5a77ab228bb2df833cd04fa6e22e8ba93dd6dd 100644
--- a/core/src/main/scala/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/spark/storage/StorageLevel.scala
@@ -7,25 +7,30 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
  * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory
  * in a serialized format, and whether to replicate the RDD partitions on multiple nodes.
  * The [[spark.storage.StorageLevel$]] singleton object contains some static constants for
- * commonly useful storage levels.
+ * commonly useful storage levels. To create your own storage level object, use the factor method
+ * of the singleton object (`StorageLevel(...)`).
  */
-class StorageLevel(
-    var useDisk: Boolean,
-    var useMemory: Boolean,
-    var deserialized: Boolean,
-    var replication: Int = 1)
+class StorageLevel private(
+    private var useDisk_ : Boolean,
+    private var useMemory_ : Boolean,
+    private var deserialized_ : Boolean,
+    private var replication_ : Int = 1)
   extends Externalizable {
 
   // TODO: Also add fields for caching priority, dataset ID, and flushing.
-
-  assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes")
-
-  def this(flags: Int, replication: Int) {
+  private def this(flags: Int, replication: Int) {
     this((flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication)
   }
 
   def this() = this(false, true, false)  // For deserialization
 
+  def useDisk = useDisk_
+  def useMemory = useMemory_
+  def deserialized = deserialized_
+  def replication = replication_
+
+  assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes")
+
   override def clone(): StorageLevel = new StorageLevel(
     this.useDisk, this.useMemory, this.deserialized, this.replication)
 
@@ -43,13 +48,13 @@ class StorageLevel(
 
   def toInt: Int = {
     var ret = 0
-    if (useDisk) {
+    if (useDisk_) {
       ret |= 4
     }
-    if (useMemory) {
+    if (useMemory_) {
       ret |= 2
     }
-    if (deserialized) {
+    if (deserialized_) {
       ret |= 1
     }
     return ret
@@ -57,15 +62,15 @@ class StorageLevel(
 
   override def writeExternal(out: ObjectOutput) {
     out.writeByte(toInt)
-    out.writeByte(replication)
+    out.writeByte(replication_)
   }
 
   override def readExternal(in: ObjectInput) {
     val flags = in.readByte()
-    useDisk = (flags & 4) != 0
-    useMemory = (flags & 2) != 0
-    deserialized = (flags & 1) != 0
-    replication = in.readByte()
+    useDisk_ = (flags & 4) != 0
+    useMemory_ = (flags & 2) != 0
+    deserialized_ = (flags & 1) != 0
+    replication_ = in.readByte()
   }
 
   @throws(classOf[IOException])
@@ -75,6 +80,14 @@ class StorageLevel(
     "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication)
 
   override def hashCode(): Int = toInt * 41 + replication
+  def description : String = {
+    var result = ""
+    result += (if (useDisk) "Disk " else "")
+    result += (if (useMemory) "Memory " else "")
+    result += (if (deserialized) "Deserialized " else "Serialized")
+    result += "%sx Replicated".format(replication)
+    result
+  }
 }
 
 
@@ -91,6 +104,21 @@ object StorageLevel {
   val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false)
   val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2)
 
+  /** Create a new StorageLevel object */
+  def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) =
+    getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication))
+
+  /** Create a new StorageLevel object from its integer representation */
+  def apply(flags: Int, replication: Int) =
+    getCachedStorageLevel(new StorageLevel(flags, replication))
+
+  /** Read StorageLevel object from ObjectInput stream */
+  def apply(in: ObjectInput) = {
+    val obj = new StorageLevel()
+    obj.readExternal(in)
+    getCachedStorageLevel(obj)
+  }
+
   private[spark]
   val storageLevelCache = new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]()
 
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..dec47a9d4113b40dcd4fe25f95a021b3cc1f681f
--- /dev/null
+++ b/core/src/main/scala/spark/storage/StorageUtils.scala
@@ -0,0 +1,82 @@
+package spark.storage
+
+import spark.{Utils, SparkContext}
+import BlockManagerMasterActor.BlockStatus
+
+private[spark]
+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)
+  }
+
+  def diskUsed(blockPrefix: String = "") = {
+    blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.diskSize).
+      reduceOption(_+_).getOrElse(0l)
+  }
+
+  def memRemaining : Long = maxMem - memUsed()
+
+}
+
+case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
+  numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) {
+  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))
+  }
+}
+
+/* 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], 
+    sc: SparkContext) : Array[RDDInfo] = {
+    rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc) 
+  }
+
+  /* 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) =>
+      k.substring(0,k.lastIndexOf('_'))
+    }.mapValues(_.values.toArray)
+
+    // For each RDD, generate an RDDInfo object
+    groupedRddBlocks.map { case(rddKey, rddBlocks) =>
+
+      // Add up memory and disk sizes
+      val memSize = rddBlocks.map(_.memSize).reduce(_ + _)
+      val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _)
+
+      // Find the id of the RDD, e.g. rdd_1 => 1
+      val rddId = rddKey.split("_").last.toInt
+      // Get the friendly name for the rdd, if available.
+      val rdd = sc.persistentRdds(rddId)
+      val rddName = Option(rdd.name).getOrElse(rddKey)
+      val rddStorageLevel = rdd.getStorageLevel
+
+      RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize)
+    }.toArray
+  }
+
+  /* Removes all BlockStatus object that are not part of a block prefix */ 
+  def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus], 
+    prefix: String) : Array[StorageStatus] = {
+
+    storageStatusList.map { status =>
+      val newBlocks = status.blocks.filterKeys(_.startsWith(prefix))
+      //val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _)
+      StorageStatus(status.blockManagerId, status.maxMem, newBlocks)
+    }
+
+  }
+
+}
diff --git a/core/src/main/scala/spark/storage/ThreadingTest.scala b/core/src/main/scala/spark/storage/ThreadingTest.scala
index 689f07b9692fbf5c68b78080f85222d86fbe8582..5c406e68cb2ac5b061dcaebd101aa8c5bcb6083b 100644
--- a/core/src/main/scala/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/spark/storage/ThreadingTest.scala
@@ -75,10 +75,10 @@ private[spark] object ThreadingTest {
     System.setProperty("spark.kryoserializer.buffer.mb", "1")
     val actorSystem = ActorSystem("test")
     val serializer = new KryoSerializer
-    val masterIp: String = System.getProperty("spark.master.host", "localhost")
-    val masterPort: Int = System.getProperty("spark.master.port", "7077").toInt
-    val blockManagerMaster = new BlockManagerMaster(actorSystem, true, true, masterIp, masterPort)
-    val blockManager = new BlockManager(actorSystem, blockManagerMaster, serializer, 1024 * 1024)
+    val blockManagerMaster = new BlockManagerMaster(
+      actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+    val blockManager = new BlockManager(
+      "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024)
     val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i))
     val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue))
     producers.foreach(_.start)
diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index 6fd9aa70fcff8e6012bc4ba960e2200bdb794b90..6f551b2b9c67572f118e5fe8692b5d7ad8c45814 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/spark/util/AkkaUtils.scala
@@ -18,18 +18,25 @@ import java.util.concurrent.TimeoutException
  * Various utility classes for working with Akka.
  */
 private[spark] object AkkaUtils {
+
   /**
    * Creates an ActorSystem ready for remoting, with various Spark features. Returns both the
    * ActorSystem itself and its port (which is hard to get from Akka).
+   *
+   * Note: the `name` parameter is important, as even if a client sends a message to right
+   * host + port, if the system name is incorrect, Akka will drop the message.
    */
   def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = {
     val akkaThreads   = System.getProperty("spark.akka.threads", "4").toInt
     val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
     val akkaTimeout   = System.getProperty("spark.akka.timeout", "20").toInt
     val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
-    val akkaConf      = ConfigFactory.parseString("""
+    val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean
+    val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean
+    val akkaConf = ConfigFactory.parseString("""
       akka.daemonic = on
       akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
+      akka.stdout-loglevel = "ERROR"
       akka.actor.provider = "akka.remote.RemoteActorRefProvider"
       akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
       akka.remote.netty.hostname = "%s"
@@ -38,9 +45,11 @@ private[spark] object AkkaUtils {
       akka.remote.netty.message-frame-size = %d MiB
       akka.remote.netty.execution-pool-size = %d
       akka.actor.default-dispatcher.throughput = %d
-      """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize))
+      akka.remote.log-remote-lifecycle-events = %s
+      """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
+                 if (lifecycleEvents) "on" else "off"))
 
-    val actorSystem = ActorSystem("spark", akkaConf, getClass.getClassLoader)
+    val actorSystem = ActorSystem(name, akkaConf, getClass.getClassLoader)
 
     // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a
     // hack because Akka doesn't let you figure out the port through the public API yet.
@@ -51,7 +60,7 @@ 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. Throws a SparkException if this fails.
+   * handle requests. Returns the bound port or throws a SparkException on failure.
    */
   def startSprayServer(actorSystem: ActorSystem, ip: String, port: Int, route: Route) {
     val ioWorker    = IOExtension(actorSystem).ioBridge()
@@ -64,7 +73,7 @@ private[spark] object AkkaUtils {
     try {
       Await.result(future, timeout) match {
         case bound: HttpServer.Bound =>
-          return
+          return server
         case other: Any =>
           throw new SparkException("Failed to bind web UI to port " + port + ": " + other)
       }
diff --git a/core/src/main/scala/spark/util/CompletionIterator.scala b/core/src/main/scala/spark/util/CompletionIterator.scala
new file mode 100644
index 0000000000000000000000000000000000000000..81391837805967141fcadf678e414a0dea7c7db6
--- /dev/null
+++ b/core/src/main/scala/spark/util/CompletionIterator.scala
@@ -0,0 +1,25 @@
+package spark.util
+
+/**
+ * Wrapper around an iterator which calls a completion method after it successfully iterates through all the elements
+ */
+abstract class CompletionIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A]{
+  def next = sub.next
+  def hasNext = {
+    val r = sub.hasNext
+    if (!r) {
+      completion
+    }
+    r
+  }
+
+  def completion()
+}
+
+object CompletionIterator {
+  def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit) : CompletionIterator[A,I] = {
+    new CompletionIterator[A,I](sub) {
+      def completion() = completionFunction
+    }
+  }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/util/Distribution.scala b/core/src/main/scala/spark/util/Distribution.scala
new file mode 100644
index 0000000000000000000000000000000000000000..24738b43078740537b86dc41339d1ae159ed07bf
--- /dev/null
+++ b/core/src/main/scala/spark/util/Distribution.scala
@@ -0,0 +1,65 @@
+package spark.util
+
+import java.io.PrintStream
+
+/**
+ * Util for getting some stats from a small sample of numeric values, with some handy summary functions.
+ *
+ * Entirely in memory, not intended as a good way to compute stats over large data sets.
+ *
+ * Assumes you are giving it a non-empty set of data
+ */
+class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) {
+  require(startIdx < endIdx)
+  def this(data: Traversable[Double]) = this(data.toArray, 0, data.size)
+  java.util.Arrays.sort(data, startIdx, endIdx)
+  val length = endIdx - startIdx
+
+  val defaultProbabilities = Array(0,0.25,0.5,0.75,1.0)
+
+  /**
+   * Get the value of the distribution at the given probabilities.  Probabilities should be
+   * given from 0 to 1
+   * @param probabilities
+   */
+  def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) = {
+    probabilities.toIndexedSeq.map{p:Double => data(closestIndex(p))}
+  }
+
+  private def closestIndex(p: Double) = {
+    math.min((p * length).toInt + startIdx, endIdx - 1)
+  }
+
+  def showQuantiles(out: PrintStream = System.out) = {
+    out.println("min\t25%\t50%\t75%\tmax")
+    getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")}
+    out.println
+  }
+
+  def statCounter = StatCounter(data.slice(startIdx, endIdx))
+
+  /**
+   * print a summary of this distribution to the given PrintStream.
+   * @param out
+   */
+  def summary(out: PrintStream = System.out) {
+    out.println(statCounter)
+    showQuantiles(out)
+  }
+}
+
+object Distribution {
+
+  def apply(data: Traversable[Double]): Option[Distribution] = {
+    if (data.size > 0)
+      Some(new Distribution(data))
+    else
+      None
+  }
+
+  def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) {
+    out.println("min\t25%\t50%\t75%\tmax")
+    quantiles.foreach{q => out.print(q + "\t")}
+    out.println
+  }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala
index 19e67acd0c7423ad2175afe44571e575318a893a..dafa90671214b4d803f2775534ebee9f5325cc70 100644
--- a/core/src/main/scala/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/spark/util/MetadataCleaner.scala
@@ -4,28 +4,30 @@ import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}
 import java.util.{TimerTask, Timer}
 import spark.Logging
 
-class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging {
 
-  val delaySeconds = (System.getProperty("spark.cleanup.delay", "-100").toDouble * 60).toInt
-  val periodSeconds = math.max(10, delaySeconds / 10)
-  val timer = new Timer(name + " cleanup timer", true)
+/**
+ * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
+ */
+class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging {
+  private val delaySeconds = MetadataCleaner.getDelaySeconds
+  private val periodSeconds = math.max(10, delaySeconds / 10)
+  private val timer = new Timer(name + " cleanup timer", true)
 
-  val task = new TimerTask {
-    def run() {
+  private val task = new TimerTask {
+    override def run() {
       try {
-        if (delaySeconds > 0) {
-          cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000))
-          logInfo("Ran metadata cleaner for " + name)
-        }
+        cleanupFunc(System.currentTimeMillis() - (delaySeconds * 1000))
+        logInfo("Ran metadata cleaner for " + name)
       } catch {
         case e: Exception => logError("Error running cleanup task for " + name, e)
       }
     }
   }
-  if (periodSeconds > 0) {
-    logInfo(
-      "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds and "
-      + "period of " + periodSeconds + " secs")
+
+  if (delaySeconds > 0) {
+    logDebug(
+      "Starting metadata cleaner for " + name + " with delay of " + delaySeconds + " seconds " +
+      "and period of " + periodSeconds + " secs")
     timer.schedule(task, periodSeconds * 1000, periodSeconds * 1000)
   }
 
@@ -33,3 +35,10 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
     timer.cancel()
   }
 }
+
+
+object MetadataCleaner {
+  def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt
+  def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.ttl", delay.toString) }
+}
+
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/RateLimitedOutputStream.scala b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e3f00ea8c734bdc45efea54fc72b6b342b31bd01
--- /dev/null
+++ b/core/src/main/scala/spark/util/RateLimitedOutputStream.scala
@@ -0,0 +1,62 @@
+package spark.util
+
+import scala.annotation.tailrec
+
+import java.io.OutputStream
+import java.util.concurrent.TimeUnit._
+
+class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends OutputStream {
+  val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS)
+  val CHUNK_SIZE = 8192
+  var lastSyncTime = System.nanoTime
+  var bytesWrittenSinceSync: Long = 0
+
+  override def write(b: Int) {
+    waitToWrite(1)
+    out.write(b)
+  }
+
+  override def write(bytes: Array[Byte]) {
+    write(bytes, 0, bytes.length)
+  }
+
+  @tailrec
+  override final def write(bytes: Array[Byte], offset: Int, length: Int) {
+    val writeSize = math.min(length - offset, CHUNK_SIZE)
+    if (writeSize > 0) {
+      waitToWrite(writeSize)
+      out.write(bytes, offset, writeSize)
+      write(bytes, offset + writeSize, length)
+    }
+  }
+
+  override def flush() {
+    out.flush()
+  }
+
+  override def close() {
+    out.close()
+  }
+
+  @tailrec
+  private def waitToWrite(numBytes: Int) {
+    val now = System.nanoTime
+    val elapsedSecs = SECONDS.convert(math.max(now - lastSyncTime, 1), NANOSECONDS)
+    val rate = bytesWrittenSinceSync.toDouble / elapsedSecs
+    if (rate < bytesPerSec) {
+      // It's okay to write; just update some variables and return
+      bytesWrittenSinceSync += numBytes
+      if (now > lastSyncTime + SYNC_INTERVAL) {
+        // Sync interval has passed; let's resync
+        lastSyncTime = now
+        bytesWrittenSinceSync = numBytes
+      }
+    } else {
+      // Calculate how much time we should sleep to bring ourselves to the desired rate.
+      // Based on throttler in Kafka (https://github.com/kafka-dev/kafka/blob/master/core/src/main/scala/kafka/utils/Throttler.scala)
+      val sleepTime = MILLISECONDS.convert((bytesWrittenSinceSync / bytesPerSec - elapsedSecs), SECONDS)
+      if (sleepTime > 0) Thread.sleep(sleepTime)
+      waitToWrite(numBytes)
+    }
+  }
+}
diff --git a/core/src/main/scala/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
index 070ee19ac0753fd4140f70c8f6aa23cec102348c..188f8910da8d54f1b43a5a8d24fa08348a6d95cb 100644
--- a/core/src/main/scala/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/spark/util/TimeStampedHashMap.scala
@@ -1,16 +1,16 @@
 package spark.util
 
 import java.util.concurrent.ConcurrentHashMap
-import scala.collection.JavaConversions._
-import scala.collection.mutable.{HashMap, Map}
+import scala.collection.JavaConversions
+import scala.collection.mutable.Map
 
 /**
  * This is a custom implementation of scala.collection.mutable.Map which stores the insertion
  * time stamp along with each key-value pair. Key-value pairs that are older than a particular
- * threshold time can them be removed using the cleanup method. This is intended to be a drop-in
+ * threshold time can them be removed using the clearOldValues method. This is intended to be a drop-in
  * replacement of scala.collection.mutable.HashMap.
  */
-class TimeStampedHashMap[A, B] extends Map[A, B]() {
+class TimeStampedHashMap[A, B] extends Map[A, B]() with spark.Logging {
   val internalMap = new ConcurrentHashMap[A, (B, Long)]()
 
   def get(key: A): Option[B] = {
@@ -20,7 +20,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
 
   def iterator: Iterator[(A, B)] = {
     val jIterator = internalMap.entrySet().iterator()
-    jIterator.map(kv => (kv.getKey, kv.getValue._1))
+    JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue._1))
   }
 
   override def + [B1 >: B](kv: (A, B1)): Map[A, B1] = {
@@ -31,8 +31,10 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
   }
 
   override def - (key: A): Map[A, B] = {
-    internalMap.remove(key)
-    this
+    val newMap = new TimeStampedHashMap[A, B]
+    newMap.internalMap.putAll(this.internalMap)
+    newMap.internalMap.remove(key)
+    newMap
   }
 
   override def += (kv: (A, B)): this.type = {
@@ -56,14 +58,14 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
   }
 
   override def filter(p: ((A, B)) => Boolean): Map[A, B] = {
-    internalMap.map(kv => (kv._1, kv._2._1)).filter(p)
+    JavaConversions.asScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
   }
 
   override def empty: Map[A, B] = new TimeStampedHashMap[A, B]()
 
-  override def size(): Int = internalMap.size()
+  override def size: Int = internalMap.size
 
-  override def foreach[U](f: ((A, B)) => U): Unit = {
+  override def foreach[U](f: ((A, B)) => U) {
     val iterator = internalMap.entrySet().iterator()
     while(iterator.hasNext) {
       val entry = iterator.next()
@@ -72,11 +74,15 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() {
     }
   }
 
-  def cleanup(threshTime: Long) {
+  /**
+   * Removes old key-value pairs that have timestamp earlier than `threshTime`
+   */
+  def clearOldValues(threshTime: Long) {
     val iterator = internalMap.entrySet().iterator()
     while(iterator.hasNext) {
       val entry = iterator.next()
       if (entry.getValue._2 < threshTime) {
+        logDebug("Removing key " + entry.getKey)
         iterator.remove()
       }
     }
diff --git a/core/src/main/scala/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/spark/util/TimeStampedHashSet.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5f1cc93752420c50ee5567220a1a917c72fd0994
--- /dev/null
+++ b/core/src/main/scala/spark/util/TimeStampedHashSet.scala
@@ -0,0 +1,69 @@
+package spark.util
+
+import scala.collection.mutable.Set
+import scala.collection.JavaConversions
+import java.util.concurrent.ConcurrentHashMap
+
+
+class TimeStampedHashSet[A] extends Set[A] {
+  val internalMap = new ConcurrentHashMap[A, Long]()
+
+  def contains(key: A): Boolean = {
+    internalMap.contains(key)
+  }
+
+  def iterator: Iterator[A] = {
+    val jIterator = internalMap.entrySet().iterator()
+    JavaConversions.asScalaIterator(jIterator).map(_.getKey)
+  }
+
+  override def + (elem: A): Set[A] = {
+    val newSet = new TimeStampedHashSet[A]
+    newSet ++= this
+    newSet += elem
+    newSet
+  }
+
+  override def - (elem: A): Set[A] = {
+    val newSet = new TimeStampedHashSet[A]
+    newSet ++= this
+    newSet -= elem
+    newSet
+  }
+
+  override def += (key: A): this.type = {
+    internalMap.put(key, currentTime)
+    this
+  }
+
+  override def -= (key: A): this.type = {
+    internalMap.remove(key)
+    this
+  }
+
+  override def empty: Set[A] = new TimeStampedHashSet[A]()
+
+  override def size(): Int = internalMap.size()
+
+  override def foreach[U](f: (A) => U): Unit = {
+    val iterator = internalMap.entrySet().iterator()
+    while(iterator.hasNext) {
+      f(iterator.next.getKey)
+    }
+  }
+
+  /**
+   * Removes old values that have timestamp earlier than `threshTime`
+   */
+  def clearOldValues(threshTime: Long) {
+    val iterator = internalMap.entrySet().iterator()
+    while(iterator.hasNext) {
+      val entry = iterator.next()
+      if (entry.getValue < threshTime) {
+        iterator.remove()
+      }
+    }
+  }
+
+  private def currentTime: Long = System.currentTimeMillis()
+}
diff --git a/core/src/main/scala/spark/util/TimedIterator.scala b/core/src/main/scala/spark/util/TimedIterator.scala
new file mode 100644
index 0000000000000000000000000000000000000000..539b01f4ce47d3ff7237ca619d220aded7b04ee1
--- /dev/null
+++ b/core/src/main/scala/spark/util/TimedIterator.scala
@@ -0,0 +1,32 @@
+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/scala/spark/util/Vector.scala b/core/src/main/scala/spark/util/Vector.scala
index 03559751bc46bea21e8d455323459cc91d73ea9c..835822edb2300969ea40497e9e954f85921ffd5e 100644
--- a/core/src/main/scala/spark/util/Vector.scala
+++ b/core/src/main/scala/spark/util/Vector.scala
@@ -11,12 +11,16 @@ class Vector(val elements: Array[Double]) extends Serializable {
     return Vector(length, i => this(i) + other(i))
   }
 
+  def add(other: Vector) = this + other
+
   def - (other: Vector): Vector = {
     if (length != other.length)
       throw new IllegalArgumentException("Vectors of different length")
     return Vector(length, i => this(i) - other(i))
   }
 
+  def subtract(other: Vector) = this - other
+
   def dot(other: Vector): Double = {
     if (length != other.length)
       throw new IllegalArgumentException("Vectors of different length")
@@ -61,10 +65,16 @@ class Vector(val elements: Array[Double]) extends Serializable {
     this
   }
 
+  def addInPlace(other: Vector) = this +=other
+
   def * (scale: Double): Vector = Vector(length, i => this(i) * scale)
 
+  def multiply (d: Double) = this * d
+
   def / (d: Double): Vector = this * (1 / d)
 
+  def divide (d: Double) = this / d
+
   def unary_- = this * -1
 
   def sum = elements.reduceLeft(_ + _)
diff --git a/core/src/main/twirl/spark/deploy/common/layout.scala.html b/core/src/main/twirl/spark/common/layout.scala.html
similarity index 100%
rename from core/src/main/twirl/spark/deploy/common/layout.scala.html
rename to core/src/main/twirl/spark/common/layout.scala.html
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
new file mode 100644
index 0000000000000000000000000000000000000000..301a7e212495d5fd9454be3f8620d41e34a58e24
--- /dev/null
+++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html
@@ -0,0 +1,40 @@
+@(app: spark.deploy.master.ApplicationInfo)
+
+@spark.common.html.layout(title = "Application Details") {
+      
+  <!-- Application Details -->
+  <div class="row">
+    <div class="span12">
+      <ul class="unstyled">
+        <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) {
+
+          } else {
+            , @app.coresLeft
+          }
+          )
+        </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>
+      </ul>
+    </div>
+  </div>
+  
+  <hr/>
+  
+  <!-- Executors -->
+  <div class="row">
+    <div class="span12">
+      <h3> Executor Summary </h3>
+      <br/>
+      @executors_table(app.executors.values.toList)
+    </div>
+  </div>
+      
+}
diff --git a/core/src/main/twirl/spark/deploy/master/app_row.scala.html b/core/src/main/twirl/spark/deploy/master/app_row.scala.html
new file mode 100644
index 0000000000000000000000000000000000000000..feb306f35ccb58d6b0097dd72584589034364e90
--- /dev/null
+++ b/core/src/main/twirl/spark/deploy/master/app_row.scala.html
@@ -0,0 +1,20 @@
+@(app: spark.deploy.master.ApplicationInfo)
+
+@import spark.Utils
+@import spark.deploy.WebUI.formatDate
+@import spark.deploy.WebUI.formatDuration
+
+<tr>
+  <td>
+    <a href="app?appId=@(app.id)">@app.id</a>
+  </td>
+  <td>@app.desc.name</td>
+  <td>
+    @app.coresGranted
+  </td>
+  <td>@Utils.memoryMegabytesToString(app.desc.memoryPerSlave)</td>
+  <td>@formatDate(app.submitDate)</td>
+  <td>@app.desc.user</td>
+  <td>@app.state.toString()</td>
+  <td>@formatDuration(app.duration)</td>
+</tr>
diff --git a/core/src/main/twirl/spark/deploy/master/job_table.scala.html b/core/src/main/twirl/spark/deploy/master/app_table.scala.html
similarity index 74%
rename from core/src/main/twirl/spark/deploy/master/job_table.scala.html
rename to core/src/main/twirl/spark/deploy/master/app_table.scala.html
index d267d6e85e0b7acb73a120c81fe71692e59d7dce..f789cee0f16ea834cecea5a76634d310a5929d66 100644
--- a/core/src/main/twirl/spark/deploy/master/job_table.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/app_table.scala.html
@@ -1,9 +1,9 @@
-@(jobs: Array[spark.deploy.master.JobInfo])
+@(apps: Array[spark.deploy.master.ApplicationInfo])
 
 <table class="table table-bordered table-striped table-condensed sortable">
   <thead>
     <tr>
-      <th>JobID</th>
+      <th>ID</th>
       <th>Description</th>
       <th>Cores</th>
       <th>Memory per Node</th>
@@ -14,8 +14,8 @@
     </tr>
   </thead>
   <tbody>
-    @for(j <- jobs) {
-      @job_row(j)
+    @for(j <- apps) {
+      @app_row(j)
     }
   </tbody>
 </table>
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 784d692fc2f91c2faa964ed76c41dfce91876e65..d2d80fad489920d1276742cc4097b37e1a639563 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
@@ -9,7 +9,7 @@
   <td>@executor.memory</td>
   <td>@executor.state</td>
   <td>
-    <a href="@(executor.worker.webUiAddress)/log?jobId=@(executor.job.id)&executorId=@(executor.id)&logType=stdout">stdout</a>
-    <a href="@(executor.worker.webUiAddress)/log?jobId=@(executor.job.id)&executorId=@(executor.id)&logType=stderr">stderr</a>
+    <a href="@(executor.worker.webUiAddress)/log?appId=@(executor.application.id)&executorId=@(executor.id)&logType=stdout">stdout</a>
+    <a href="@(executor.worker.webUiAddress)/log?appId=@(executor.application.id)&executorId=@(executor.id)&logType=stderr">stderr</a>
   </td>
-</tr>
\ No newline at end of file
+</tr>
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 18c32e5a1f094b35c8dd0107eeadffcd8f09a63a..ac51a39a5199d4cd96636f602ba0a9e7ce8b23c0 100644
--- a/core/src/main/twirl/spark/deploy/master/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/index.scala.html
@@ -2,19 +2,19 @@
 @import spark.deploy.master._
 @import spark.Utils
 
-@spark.deploy.common.html.layout(title = "Spark Master on " + state.uri) {
-      
+@spark.common.html.layout(title = "Spark Master on " + state.host) {
+
   <!-- Cluster Details -->
   <div class="row">
     <div class="span12">
       <ul class="unstyled">
-        <li><strong>URL:</strong> spark://@(state.uri)</li>
+        <li><strong>URL:</strong> @(state.uri)</li>
         <li><strong>Workers:</strong> @state.workers.size </li>
         <li><strong>Cores:</strong> @{state.workers.map(_.cores).sum} Total,
                                     @{state.workers.map(_.coresUsed).sum} Used</li>
         <li><strong>Memory:</strong> @{Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total, 
                                      @{Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used</li>
-        <li><strong>Jobs:</strong> @state.activeJobs.size Running, @state.completedJobs.size Completed </li>
+        <li><strong>Applications:</strong> @state.activeApps.size Running, @state.completedApps.size Completed </li>
       </ul>
     </div>
   </div>
@@ -22,7 +22,7 @@
   <!-- Worker Summary -->
   <div class="row">
     <div class="span12">
-      <h3> Cluster Summary </h3>
+      <h3> Workers </h3>
       <br/>
       @worker_table(state.workers.sortBy(_.id))
     </div>
@@ -30,23 +30,23 @@
 
   <hr/>
 
-  <!-- Job Summary (Running) -->
+  <!-- App Summary (Running) -->
   <div class="row">
     <div class="span12">
-      <h3> Running Jobs </h3>
+      <h3> Running Applications </h3>
       <br/>
-      @job_table(state.activeJobs.sortBy(_.startTime).reverse)
+      @app_table(state.activeApps.sortBy(_.startTime).reverse)
     </div>
   </div>
 
   <hr/>
 
-  <!-- Job Summary (Completed) -->
+  <!-- App Summary (Completed) -->
   <div class="row">
     <div class="span12">
-      <h3> Completed Jobs </h3>
+      <h3> Completed Applications </h3>
       <br/>
-      @job_table(state.completedJobs.sortBy(_.endTime).reverse)
+      @app_table(state.completedApps.sortBy(_.endTime).reverse)
     </div>
   </div>
 
diff --git a/core/src/main/twirl/spark/deploy/master/job_details.scala.html b/core/src/main/twirl/spark/deploy/master/job_details.scala.html
deleted file mode 100644
index dcf41c28f26f56fa289f7ef7a71c9067b47d1baf..0000000000000000000000000000000000000000
--- a/core/src/main/twirl/spark/deploy/master/job_details.scala.html
+++ /dev/null
@@ -1,40 +0,0 @@
-@(job: spark.deploy.master.JobInfo)
-
-@spark.deploy.common.html.layout(title = "Job Details") {
-      
-  <!-- Job Details -->
-  <div class="row">
-    <div class="span12">
-      <ul class="unstyled">
-        <li><strong>ID:</strong> @job.id</li>
-        <li><strong>Description:</strong> @job.desc.name</li>
-        <li><strong>User:</strong> @job.desc.user</li>
-        <li><strong>Cores:</strong> 
-          @job.desc.cores 
-          (@job.coresGranted Granted 
-          @if(job.desc.cores == Integer.MAX_VALUE) {
-
-          } else {
-            , @job.coresLeft
-          }
-          )
-        </li>
-        <li><strong>Memory per Slave:</strong> @job.desc.memoryPerSlave</li>
-        <li><strong>Submit Date:</strong> @job.submitDate</li>
-        <li><strong>State:</strong> @job.state</li>
-      </ul>
-    </div>
-  </div>
-  
-  <hr/>
-  
-  <!-- Executors -->
-  <div class="row">
-    <div class="span12">
-      <h3> Executor Summary </h3>
-      <br/>
-      @executors_table(job.executors.values.toList)
-    </div>
-  </div>
-      
-}
diff --git a/core/src/main/twirl/spark/deploy/master/job_row.scala.html b/core/src/main/twirl/spark/deploy/master/job_row.scala.html
deleted file mode 100644
index 7c466a6a2ce4d4ec722cd2435b7d084b35dc5d7f..0000000000000000000000000000000000000000
--- a/core/src/main/twirl/spark/deploy/master/job_row.scala.html
+++ /dev/null
@@ -1,20 +0,0 @@
-@(job: spark.deploy.master.JobInfo)
-
-@import spark.Utils
-@import spark.deploy.WebUI.formatDate
-@import spark.deploy.WebUI.formatDuration
-
-<tr>
-  <td>
-    <a href="job?jobId=@(job.id)">@job.id</a>
-  </td>
-  <td>@job.desc.name</td>
-  <td>
-    @job.coresGranted
-  </td>
-  <td>@Utils.memoryMegabytesToString(job.desc.memoryPerSlave)</td>
-  <td>@formatDate(job.submitDate)</td>
-  <td>@job.desc.user</td>
-  <td>@job.state.toString()</td>
-  <td>@formatDuration(job.duration)</td>
-</tr>
diff --git a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
index ea9542461e5f736b41829f9bea8c07521dcc79f9..dad0a89080f2528848190b6803e7727d5bf932b8 100644
--- a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
@@ -8,13 +8,13 @@
   <td>@Utils.memoryMegabytesToString(executor.memory)</td>
   <td>
     <ul class="unstyled">
-      <li><strong>ID:</strong> @executor.jobId</li>
-      <li><strong>Name:</strong> @executor.jobDesc.name</li>
-      <li><strong>User:</strong> @executor.jobDesc.user</li>
+      <li><strong>ID:</strong> @executor.appId</li>
+      <li><strong>Name:</strong> @executor.appDesc.name</li>
+      <li><strong>User:</strong> @executor.appDesc.user</li>
     </ul>
   </td>
   <td>
-    <a href="log?jobId=@(executor.jobId)&executorId=@(executor.execId)&logType=stdout">stdout</a>
-    <a href="log?jobId=@(executor.jobId)&executorId=@(executor.execId)&logType=stderr">stderr</a>
+    <a href="log?appId=@(executor.appId)&executorId=@(executor.execId)&logType=stdout">stdout</a>
+    <a href="log?appId=@(executor.appId)&executorId=@(executor.execId)&logType=stderr">stderr</a>
   </td>
 </tr>
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 b247307dab06793be7877bee00ffd1255bd4da8c..c39f769a7387f96113c3f088a88fd6d1ac19351e 100644
--- a/core/src/main/twirl/spark/deploy/worker/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html
@@ -1,9 +1,8 @@
 @(worker: spark.deploy.WorkerState)
-
 @import spark.Utils
 
-@spark.deploy.common.html.layout(title = "Spark Worker on " + worker.uri) {
-      
+@spark.common.html.layout(title = "Spark Worker on " + worker.host) {
+
   <!-- Worker Details -->
   <div class="row">
     <div class="span12">
@@ -11,12 +10,12 @@
         <li><strong>ID:</strong> @worker.workerId</li>
         <li><strong>
           Master URL:</strong> @worker.masterUrl 
-          (WebUI at <a href="@worker.masterWebUiUrl">@worker.masterWebUiUrl</a>)
         </li>
         <li><strong>Cores:</strong> @worker.cores (@worker.coresUsed Used)</li>
         <li><strong>Memory:</strong> @{Utils.memoryMegabytesToString(worker.memory)}
           (@{Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</li>
       </ul>
+      <p><a href="@worker.masterWebUiUrl">Back to Master</a></p>
     </div>
   </div>
 
diff --git a/core/src/main/twirl/spark/storage/index.scala.html b/core/src/main/twirl/spark/storage/index.scala.html
new file mode 100644
index 0000000000000000000000000000000000000000..2b337f61339b213bea1ac62941363708373e4c58
--- /dev/null
+++ b/core/src/main/twirl/spark/storage/index.scala.html
@@ -0,0 +1,40 @@
+@(maxMem: Long, remainingMem: Long, diskSpaceUsed: Long, rdds: Array[spark.storage.RDDInfo], storageStatusList: Array[spark.storage.StorageStatus])
+@import spark.Utils
+
+@spark.common.html.layout(title = "Storage Dashboard") {
+  
+  <!-- High-Level Information -->
+  <div class="row">
+    <div class="span12">
+      <ul class="unstyled">
+        <li><strong>Memory:</strong> 
+          @{Utils.memoryBytesToString(maxMem - remainingMem)} Used 
+          (@{Utils.memoryBytesToString(remainingMem)} Available) </li>
+        <li><strong>Disk:</strong> @{Utils.memoryBytesToString(diskSpaceUsed)} Used </li>
+      </ul>
+    </div>
+  </div>
+
+  <hr/>
+
+  <!-- RDD Summary -->
+  <div class="row">
+    <div class="span12">
+      <h3> RDD Summary </h3>
+      <br/>
+       @rdd_table(rdds)
+    </div>
+  </div>
+
+  <hr/>
+
+  <!-- Worker Summary -->
+  <div class="row">
+    <div class="span12">
+      <h3> Worker Summary </h3>
+      <br/>
+       @worker_table(storageStatusList)
+    </div>
+  </div>
+
+}
\ No newline at end of file
diff --git a/core/src/main/twirl/spark/storage/rdd.scala.html b/core/src/main/twirl/spark/storage/rdd.scala.html
new file mode 100644
index 0000000000000000000000000000000000000000..d85addeb1755e77fa0c7ee947b8e6892aa3e29d4
--- /dev/null
+++ b/core/src/main/twirl/spark/storage/rdd.scala.html
@@ -0,0 +1,81 @@
+@(rddInfo: spark.storage.RDDInfo, storageStatusList: Array[spark.storage.StorageStatus])
+@import spark.Utils
+
+@spark.common.html.layout(title = "RDD Info ") {
+  
+  <!-- High-Level Information -->
+  <div class="row">
+    <div class="span12">
+      <ul class="unstyled">
+        <li>
+          <strong>Storage Level:</strong> 
+          @(rddInfo.storageLevel.description)
+        <li>
+          <strong>Cached Partitions:</strong>
+          @(rddInfo.numCachedPartitions)
+        </li>
+        <li>
+          <strong>Total Partitions:</strong>
+          @(rddInfo.numPartitions)
+        </li>
+        <li>
+          <strong>Memory Size:</strong>
+          @{Utils.memoryBytesToString(rddInfo.memSize)}
+        </li>
+        <li>
+          <strong>Disk Size:</strong>
+          @{Utils.memoryBytesToString(rddInfo.diskSize)}
+        </li>
+      </ul>
+    </div>
+  </div>
+
+  <hr/>
+
+  <!-- RDD Summary -->
+  <div class="row">
+    <div class="span12">
+      <h3> RDD Summary </h3>
+      <br/>
+      
+
+      <!-- Block Table Summary -->
+      <table class="table table-bordered table-striped table-condensed sortable">
+        <thead>
+          <tr>
+            <th>Block Name</th>
+            <th>Storage Level</th>
+            <th>Size in Memory</th>
+            <th>Size on Disk</th>
+          </tr>
+        </thead>
+        <tbody>
+          @storageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1).map { case (k,v) => 
+            <tr>
+              <td>@k</td>
+              <td>
+                 @(v.storageLevel.description)
+              </td>
+              <td>@{Utils.memoryBytesToString(v.memSize)}</td>
+              <td>@{Utils.memoryBytesToString(v.diskSize)}</td>
+            </tr>
+          }
+        </tbody>
+      </table>
+
+
+    </div>
+  </div>
+
+  <hr/>
+
+  <!-- Worker Table -->
+  <div class="row">
+    <div class="span12">
+      <h3> Worker Summary </h3>
+      <br/>
+       @worker_table(storageStatusList, "rdd_" + rddInfo.id )
+    </div>
+  </div>
+
+}
\ No newline at end of file
diff --git a/core/src/main/twirl/spark/storage/rdd_table.scala.html b/core/src/main/twirl/spark/storage/rdd_table.scala.html
new file mode 100644
index 0000000000000000000000000000000000000000..a51e64aed007292782fa3df720a7ebf2130c76f8
--- /dev/null
+++ b/core/src/main/twirl/spark/storage/rdd_table.scala.html
@@ -0,0 +1,32 @@
+@(rdds: Array[spark.storage.RDDInfo])
+@import spark.Utils
+
+<table class="table table-bordered table-striped table-condensed sortable">
+  <thead>
+    <tr>
+      <th>RDD Name</th>
+      <th>Storage Level</th>
+      <th>Cached Partitions</th>
+      <th>Fraction Partitions Cached</th>
+      <th>Size in Memory</th>
+      <th>Size on Disk</th>
+    </tr>
+  </thead>
+  <tbody>
+    @for(rdd <- rdds) {
+      <tr>
+        <td>
+          <a href="rdd?id=@(rdd.id)">
+            @rdd.name
+          </a>
+        </td>
+        <td>@(rdd.storageLevel.description)
+        </td>
+        <td>@rdd.numCachedPartitions</td>
+        <td>@(rdd.numCachedPartitions / rdd.numPartitions.toDouble)</td>
+        <td>@{Utils.memoryBytesToString(rdd.memSize)}</td>
+        <td>@{Utils.memoryBytesToString(rdd.diskSize)}</td>
+      </tr>
+    }
+  </tbody>
+</table>
\ No newline at end of file
diff --git a/core/src/main/twirl/spark/storage/worker_table.scala.html b/core/src/main/twirl/spark/storage/worker_table.scala.html
new file mode 100644
index 0000000000000000000000000000000000000000..d54b8de4cc81394149cceac0d52b965483e95a7c
--- /dev/null
+++ b/core/src/main/twirl/spark/storage/worker_table.scala.html
@@ -0,0 +1,24 @@
+@(workersStatusList: Array[spark.storage.StorageStatus], prefix: String = "")
+@import spark.Utils
+
+<table class="table table-bordered table-striped table-condensed sortable">
+  <thead>
+    <tr>
+      <th>Host</th>
+      <th>Memory Usage</th>
+      <th>Disk Usage</th>
+    </tr>
+  </thead>
+  <tbody>
+    @for(status <- workersStatusList) {
+      <tr>
+        <td>@(status.blockManagerId.ip + ":" + status.blockManagerId.port)</td>
+        <td>
+          @(Utils.memoryBytesToString(status.memUsed(prefix)))
+          (@(Utils.memoryBytesToString(status.memRemaining)) Total Available)
+        </td>
+        <td>@(Utils.memoryBytesToString(status.diskUsed(prefix)))</td>
+    </tr>
+    }
+  </tbody>
+</table>
\ No newline at end of file
diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties
index 4c99e450bccccff6e67cbd38e0566553764adb2d..6ec89c01840b6935c061d30820e72cf36aa3314d 100644
--- a/core/src/test/resources/log4j.properties
+++ b/core/src/test/resources/log4j.properties
@@ -1,8 +1,8 @@
-# Set everything to be logged to the console
+# Set everything to be logged to the file core/target/unit-tests.log 
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false
-log4j.appender.file.file=spark-tests.log
+log4j.appender.file.file=core/target/unit-tests.log
 log4j.appender.file.layout=org.apache.log4j.PatternLayout
 log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
 
diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala
index 9f5335978f35a8b8efe4a2bc0d9728a74f67c11d..f59334a033d7a00f3b00c4ee9b20713839ee0f39 100644
--- a/core/src/test/scala/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/spark/AccumulatorSuite.scala
@@ -1,6 +1,5 @@
 package spark
 
-import org.scalatest.BeforeAndAfter
 import org.scalatest.FunSuite
 import org.scalatest.matchers.ShouldMatchers
 import collection.mutable
@@ -9,9 +8,8 @@ import scala.math.exp
 import scala.math.signum
 import spark.SparkContext._
 
-class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
+class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext {
 
-  var sc: SparkContext = null
 
   implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] {
     def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = {
@@ -27,15 +25,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
     }
   }
 
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
-
   test ("basic accumulation"){
     sc = new SparkContext("local", "test")
     val acc : Accumulator[Int] = sc.accumulator(0)
@@ -43,6 +32,12 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
     val d = sc.parallelize(1 to 20)
     d.foreach{x => acc += x}
     acc.value should be (210)
+
+
+    val longAcc = sc.accumulator(0l)
+    val maxInt = Integer.MAX_VALUE.toLong
+    d.foreach{x => longAcc += maxInt + x}
+    longAcc.value should be (210l + maxInt * 20)
   }
 
   test ("value not assignable from tasks") {
@@ -66,10 +61,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
       for (i <- 1 to maxI) {
         v should contain(i)
       }
-      sc.stop()
-      sc = null
-      // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-      System.clearProperty("spark.master.port")
+      resetSparkContext()
     }
   }
 
@@ -84,10 +76,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
           x => acc.value += x
         }
       } should produce [SparkException]
-      sc.stop()
-      sc = null
-      // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-      System.clearProperty("spark.master.port")
+      resetSparkContext()
     }
   }
 
@@ -113,10 +102,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
         bufferAcc.value should contain(i)
         mapAcc.value should contain (i -> i.toString)
       }
-      sc.stop()
-      sc = null
-      // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-      System.clearProperty("spark.master.port")
+      resetSparkContext()
     }
   }
 
@@ -131,8 +117,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
         x => acc.localValue ++= x
       }
       acc.value should be ( (0 to maxI).toSet)
-      sc.stop()
-      sc = null
+      resetSparkContext()
     }
   }
 
diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala
index 2d3302f0aa2288a7f9970d616855ae1ece810401..362a31fb0d9715d9c0ef71a29f69d939b27c00d2 100644
--- a/core/src/test/scala/spark/BroadcastSuite.scala
+++ b/core/src/test/scala/spark/BroadcastSuite.scala
@@ -1,20 +1,8 @@
 package spark
 
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 
-class BroadcastSuite extends FunSuite with BeforeAndAfter {
-  
-  var sc: SparkContext = _
-  
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class BroadcastSuite extends FunSuite with LocalSparkContext {
   
   test("basic broadcast") {
     sc = new SparkContext("local", "test")
diff --git a/core/src/test/scala/spark/CacheTrackerSuite.scala b/core/src/test/scala/spark/CacheTrackerSuite.scala
deleted file mode 100644
index d0c2bd47fc02b0fad16722db332fc3c80e69de1a..0000000000000000000000000000000000000000
--- a/core/src/test/scala/spark/CacheTrackerSuite.scala
+++ /dev/null
@@ -1,130 +0,0 @@
-package spark
-
-import org.scalatest.FunSuite
-
-import scala.collection.mutable.HashMap
-
-import akka.actor._
-import scala.concurrent.{Await, Future}
-import akka.remote._
-import scala.concurrent.duration.Duration
-import akka.util.Timeout
-import scala.concurrent.duration._
-
-class CacheTrackerSuite extends FunSuite {
-  // Send a message to an actor and wait for a reply, in a blocking manner
-  private def ask(actor: ActorRef, message: Any): Any = {
-    try {
-      val timeout = 10.seconds
-      val future: Future[Any] = akka.pattern.ask(actor, message)(timeout)
-      Await.result(future, timeout)
-    } catch {
-      case e: Exception =>
-        throw new SparkException("Error communicating with actor", e)
-    }
-  }
-
-  test("CacheTrackerActor slave initialization & cache status") {
-    //System.setProperty("spark.master.port", "1345")
-    val initialSize = 2L << 20
-
-    val actorSystem = ActorSystem("test")
-    val tracker = actorSystem.actorOf(Props[CacheTrackerActor])
-
-    assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true)
-
-    assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 0L)))
-
-    assert(ask(tracker, StopCacheTracker) === true)
-    
-    actorSystem.shutdown()
-    actorSystem.awaitTermination()
-  }
-
-  test("RegisterRDD") {
-    //System.setProperty("spark.master.port", "1345")
-    val initialSize = 2L << 20
-
-    val actorSystem = ActorSystem("test")
-    val tracker = actorSystem.actorOf(Props[CacheTrackerActor])
-
-    assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true)
-
-    assert(ask(tracker, RegisterRDD(1, 3)) === true)
-    assert(ask(tracker, RegisterRDD(2, 1)) === true)
-
-    assert(getCacheLocations(tracker) === Map(1 -> List(Nil, Nil, Nil), 2 -> List(Nil)))
-
-    assert(ask(tracker, StopCacheTracker) === true)
-    
-    actorSystem.shutdown()
-    actorSystem.awaitTermination()
-  }
-
-  test("AddedToCache") {
-    //System.setProperty("spark.master.port", "1345")
-    val initialSize = 2L << 20
-
-    val actorSystem = ActorSystem("test")
-    val tracker = actorSystem.actorOf(Props[CacheTrackerActor])
-
-    assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true)
-
-    assert(ask(tracker, RegisterRDD(1, 2)) === true)
-    assert(ask(tracker, RegisterRDD(2, 1)) === true)
-
-    assert(ask(tracker, AddedToCache(1, 0, "host001", 2L << 15)) === true)
-    assert(ask(tracker, AddedToCache(1, 1, "host001", 2L << 11)) === true)
-    assert(ask(tracker, AddedToCache(2, 0, "host001", 3L << 10)) === true)
-
-    assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 72704L)))
-
-    assert(getCacheLocations(tracker) === 
-      Map(1 -> List(List("host001"), List("host001")), 2 -> List(List("host001"))))
-
-    assert(ask(tracker, StopCacheTracker) === true)
-    
-    actorSystem.shutdown()
-    actorSystem.awaitTermination()
-  }
-
-  test("DroppedFromCache") {
-    //System.setProperty("spark.master.port", "1345")
-    val initialSize = 2L << 20
-
-    val actorSystem = ActorSystem("test")
-    val tracker = actorSystem.actorOf(Props[CacheTrackerActor])
-
-    assert(ask(tracker, SlaveCacheStarted("host001", initialSize)) === true)
-
-    assert(ask(tracker, RegisterRDD(1, 2)) === true)
-    assert(ask(tracker, RegisterRDD(2, 1)) === true)
-
-    assert(ask(tracker, AddedToCache(1, 0, "host001", 2L << 15)) === true)
-    assert(ask(tracker, AddedToCache(1, 1, "host001", 2L << 11)) === true)
-    assert(ask(tracker, AddedToCache(2, 0, "host001", 3L << 10)) === true)
-
-    assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 72704L)))
-    assert(getCacheLocations(tracker) ===
-      Map(1 -> List(List("host001"), List("host001")), 2 -> List(List("host001"))))
-
-    assert(ask(tracker, DroppedFromCache(1, 1, "host001", 2L << 11)) === true)
-
-    assert(ask(tracker, GetCacheStatus) === Seq(("host001", 2097152L, 68608L)))
-    assert(getCacheLocations(tracker) ===
-      Map(1 -> List(List("host001"),List()), 2 -> List(List("host001"))))
-
-    assert(ask(tracker, StopCacheTracker) === true)
-    
-    actorSystem.shutdown()
-    actorSystem.awaitTermination()
-  }
-
-  /**
-   * Helper function to get cacheLocations from CacheTracker
-   */
-  def getCacheLocations(tracker: ActorRef): HashMap[Int, List[List[String]]] = {
-    val answer = ask(tracker, GetCacheLocations).asInstanceOf[HashMap[Int, Array[List[String]]]]
-    answer.map { case (i, arr) => (i, arr.toList) }
-  }
-}
diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ca385972fb2ebe3add71881f18c3edc9761077d8
--- /dev/null
+++ b/core/src/test/scala/spark/CheckpointSuite.scala
@@ -0,0 +1,365 @@
+package spark
+
+import org.scalatest.FunSuite
+import java.io.File
+import spark.rdd._
+import spark.SparkContext._
+import storage.StorageLevel
+
+class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
+  initLogging()
+
+  var checkpointDir: File = _
+  val partitioner = new HashPartitioner(2)
+
+  override def beforeEach() {
+    super.beforeEach()
+    checkpointDir = File.createTempFile("temp", "")
+    checkpointDir.delete()
+    sc = new SparkContext("local", "test")
+    sc.setCheckpointDir(checkpointDir.toString)
+  }
+
+  override def afterEach() {
+    super.afterEach()
+    if (checkpointDir != null) {
+      checkpointDir.delete()
+    }
+  }
+
+  test("RDDs with one-to-one dependencies") {
+    testCheckpointing(_.map(x => x.toString))
+    testCheckpointing(_.flatMap(x => 1 to x))
+    testCheckpointing(_.filter(_ % 2 == 0))
+    testCheckpointing(_.sample(false, 0.5, 0))
+    testCheckpointing(_.glom())
+    testCheckpointing(_.mapPartitions(_.map(_.toString)))
+    testCheckpointing(r => new MapPartitionsWithIndexRDD(r,
+      (i: Int, iter: Iterator[Int]) => iter.map(_.toString), false ))
+    testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
+    testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
+    testCheckpointing(_.pipe(Seq("cat")))
+  }
+
+  test("ParallelCollection") {
+    val parCollection = sc.makeRDD(1 to 4, 2)
+    val numPartitions = parCollection.partitions.size
+    parCollection.checkpoint()
+    assert(parCollection.dependencies === Nil)
+    val result = parCollection.collect()
+    assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result)
+    assert(parCollection.dependencies != Nil)
+    assert(parCollection.partitions.length === numPartitions)
+    assert(parCollection.partitions.toList === parCollection.checkpointData.get.getPartitions.toList)
+    assert(parCollection.collect() === result)
+  }
+
+  test("BlockRDD") {
+    val blockId = "id"
+    val blockManager = SparkEnv.get.blockManager
+    blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY)
+    val blockRDD = new BlockRDD[String](sc, Array(blockId))
+    val numPartitions = blockRDD.partitions.size
+    blockRDD.checkpoint()
+    val result = blockRDD.collect()
+    assert(sc.checkpointFile[String](blockRDD.getCheckpointFile.get).collect() === result)
+    assert(blockRDD.dependencies != Nil)
+    assert(blockRDD.partitions.length === numPartitions)
+    assert(blockRDD.partitions.toList === blockRDD.checkpointData.get.getPartitions.toList)
+    assert(blockRDD.collect() === result)
+  }
+
+  test("ShuffledRDD") {
+    testCheckpointing(rdd => {
+      // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
+      new ShuffledRDD(rdd.map(x => (x % 2, 1)), partitioner)
+    })
+  }
+
+  test("UnionRDD") {
+    def otherRDD = sc.makeRDD(1 to 10, 1)
+
+    // Test whether the size of UnionRDDPartitions reduce in size after parent RDD is checkpointed.
+    // Current implementation of UnionRDD has transient reference to parent RDDs,
+    // so only the partitions will reduce in serialized size, not the RDD.
+    testCheckpointing(_.union(otherRDD), false, true)
+    testParentCheckpointing(_.union(otherRDD), false, true)
+  }
+
+  test("CartesianRDD") {
+    def otherRDD = sc.makeRDD(1 to 10, 1)
+    testCheckpointing(new CartesianRDD(sc, _, otherRDD))
+
+    // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
+    // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
+    // so only the RDD will reduce in serialized size, not the partitions.
+    testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false)
+
+    // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after
+    // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
+    // Note that this test is very specific to the current implementation of CartesianRDD.
+    val ones = sc.makeRDD(1 to 100, 10).map(x => x)
+    ones.checkpoint() // checkpoint that MappedRDD
+    val cartesian = new CartesianRDD(sc, ones, ones)
+    val splitBeforeCheckpoint =
+      serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition])
+    cartesian.count() // do the checkpointing
+    val splitAfterCheckpoint =
+      serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition])
+    assert(
+      (splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) &&
+        (splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2),
+      "CartesianRDD.parents not updated after parent RDD checkpointed"
+    )
+  }
+
+  test("CoalescedRDD") {
+    testCheckpointing(_.coalesce(2))
+
+    // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
+    // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
+    // so only the RDD will reduce in serialized size, not the partitions.
+    testParentCheckpointing(_.coalesce(2), true, false)
+
+    // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) after
+    // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
+    // Note that this test is very specific to the current implementation of CoalescedRDDPartitions
+    val ones = sc.makeRDD(1 to 100, 10).map(x => x)
+    ones.checkpoint() // checkpoint that MappedRDD
+    val coalesced = new CoalescedRDD(ones, 2)
+    val splitBeforeCheckpoint =
+      serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition])
+    coalesced.count() // do the checkpointing
+    val splitAfterCheckpoint =
+      serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition])
+    assert(
+      splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head,
+      "CoalescedRDDPartition.parents not updated after parent RDD checkpointed"
+    )
+  }
+
+  test("CoGroupedRDD") {
+    val longLineageRDD1 = generateLongLineageRDDForCoGroupedRDD()
+    testCheckpointing(rdd => {
+      CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner)
+    }, false, true)
+
+    val longLineageRDD2 = generateLongLineageRDDForCoGroupedRDD()
+    testParentCheckpointing(rdd => {
+      CheckpointSuite.cogroup(
+        longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner)
+    }, false, true)
+  }
+
+  test("ZippedRDD") {
+    testCheckpointing(
+      rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
+
+    // Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed
+    // Current implementation of ZippedRDDPartitions has transient references to parent RDDs,
+    // so only the RDD will reduce in serialized size, not the partitions.
+    testParentCheckpointing(
+      rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
+  }
+
+  test("CheckpointRDD with zero partitions") {
+    val rdd = new BlockRDD[Int](sc, Array[String]())
+    assert(rdd.partitions.size === 0)
+    assert(rdd.isCheckpointed === false)
+    rdd.checkpoint()
+    assert(rdd.count() === 0)
+    assert(rdd.isCheckpointed === true)
+    assert(rdd.partitions.size === 0)
+  }
+
+  /**
+   * Test checkpointing of the final RDD generated by the given operation. By default,
+   * this method tests whether the size of serialized RDD has reduced after checkpointing or not.
+   * It can also test whether the size of serialized RDD partitions has reduced after checkpointing or
+   * not, but this is not done by default as usually the partitions do not refer to any RDD and
+   * therefore never store the lineage.
+   */
+  def testCheckpointing[U: ClassManifest](
+      op: (RDD[Int]) => RDD[U],
+      testRDDSize: Boolean = true,
+      testRDDPartitionSize: Boolean = false
+    ) {
+    // Generate the final RDD using given RDD operation
+    val baseRDD = generateLongLineageRDD()
+    val operatedRDD = op(baseRDD)
+    val parentRDD = operatedRDD.dependencies.headOption.orNull
+    val rddType = operatedRDD.getClass.getSimpleName
+    val numPartitions = operatedRDD.partitions.length
+
+    // Find serialized sizes before and after the checkpoint
+    val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+    operatedRDD.checkpoint()
+    val result = operatedRDD.collect()
+    val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+
+    // Test whether the checkpoint file has been created
+    assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result)
+
+    // Test whether dependencies have been changed from its earlier parent RDD
+    assert(operatedRDD.dependencies.head.rdd != parentRDD)
+
+    // Test whether the partitions have been changed to the new Hadoop partitions
+    assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList)
+
+    // Test whether the number of partitions is same as before
+    assert(operatedRDD.partitions.length === numPartitions)
+
+    // Test whether the data in the checkpointed RDD is same as original
+    assert(operatedRDD.collect() === result)
+
+    // Test whether serialized size of the RDD has reduced. If the RDD
+    // does not have any dependency to another RDD (e.g., ParallelCollection,
+    // ShuffleRDD with ShuffleDependency), it may not reduce in size after checkpointing.
+    if (testRDDSize) {
+      logInfo("Size of " + rddType +
+        "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
+      assert(
+        rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
+        "Size of " + rddType + " did not reduce after checkpointing " +
+          "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
+      )
+    }
+
+    // Test whether serialized size of the partitions has reduced. If the partitions
+    // do not have any non-transient reference to another RDD or another RDD's partitions, it
+    // does not refer to a lineage and therefore may not reduce in size after checkpointing.
+    // However, if the original partitions before checkpointing do refer to a parent RDD, the partitions
+    // must be forgotten after checkpointing (to remove all reference to parent RDDs) and
+    // replaced with the HadooPartitions of the checkpointed RDD.
+    if (testRDDPartitionSize) {
+      logInfo("Size of " + rddType + " partitions "
+        + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]")
+      assert(
+        splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
+        "Size of " + rddType + " partitions did not reduce after checkpointing " +
+          "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
+      )
+    }
+  }
+
+  /**
+   * Test whether checkpointing of the parent of the generated RDD also
+   * truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent
+   * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed,
+   * this RDD will remember the partitions and therefore potentially the whole lineage.
+   */
+  def testParentCheckpointing[U: ClassManifest](
+      op: (RDD[Int]) => RDD[U],
+      testRDDSize: Boolean,
+      testRDDPartitionSize: Boolean
+    ) {
+    // Generate the final RDD using given RDD operation
+    val baseRDD = generateLongLineageRDD()
+    val operatedRDD = op(baseRDD)
+    val parentRDD = operatedRDD.dependencies.head.rdd
+    val rddType = operatedRDD.getClass.getSimpleName
+    val parentRDDType = parentRDD.getClass.getSimpleName
+
+    // Get the partitions and dependencies of the parent in case they're lazily computed
+    parentRDD.dependencies
+    parentRDD.partitions
+
+    // Find serialized sizes before and after the checkpoint
+    val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+    parentRDD.checkpoint()  // checkpoint the parent RDD, not the generated one
+    val result = operatedRDD.collect()
+    val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+
+    // Test whether the data in the checkpointed RDD is same as original
+    assert(operatedRDD.collect() === result)
+
+    // Test whether serialized size of the RDD has reduced because of its parent being
+    // checkpointed. If this RDD or its parent RDD do not have any dependency
+    // to another RDD (e.g., ParallelCollection, ShuffleRDD with ShuffleDependency), it may
+    // not reduce in size after checkpointing.
+    if (testRDDSize) {
+      assert(
+        rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
+        "Size of " + rddType + " did not reduce after checkpointing parent " + parentRDDType +
+          "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
+      )
+    }
+
+    // Test whether serialized size of the partitions has reduced because of its parent being
+    // checkpointed. If the partitions do not have any non-transient reference to another RDD
+    // or another RDD's partitions, it does not refer to a lineage and therefore may not reduce
+    // in size after checkpointing. However, if the partitions do refer to the *partitions* of a parent
+    // RDD, then these partitions must update reference to the parent RDD partitions as the parent RDD's
+    // partitions must have changed after checkpointing.
+    if (testRDDPartitionSize) {
+      assert(
+        splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
+        "Size of " + rddType + " partitions did not reduce after checkpointing parent " + parentRDDType +
+          "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
+      )
+    }
+
+  }
+
+  /**
+   * Generate an RDD with a long lineage of one-to-one dependencies.
+   */
+  def generateLongLineageRDD(): RDD[Int] = {
+    var rdd = sc.makeRDD(1 to 100, 4)
+    for (i <- 1 to 50) {
+      rdd = rdd.map(x => x + 1)
+    }
+    rdd
+  }
+
+  /**
+   * Generate an RDD with a long lineage specifically for CoGroupedRDD.
+   * A CoGroupedRDD can have a long lineage only one of its parents have a long lineage
+   * and narrow dependency with this RDD. This method generate such an RDD by a sequence
+   * of cogroups and mapValues which creates a long lineage of narrow dependencies.
+   */
+  def generateLongLineageRDDForCoGroupedRDD() = {
+    val add = (x: (Seq[Int], Seq[Int])) => (x._1 ++ x._2).reduce(_ + _)
+
+    def ones: RDD[(Int, Int)] = sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
+
+    var cogrouped: RDD[(Int, (Seq[Int], Seq[Int]))] = ones.cogroup(ones)
+    for(i <- 1 to 10) {
+      cogrouped = cogrouped.mapValues(add).cogroup(ones)
+    }
+    cogrouped.mapValues(add)
+  }
+
+  /**
+   * Get serialized sizes of the RDD and its partitions, in order to test whether the size shrinks
+   * upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint.
+   */
+  def getSerializedSizes(rdd: RDD[_]): (Int, Int) = {
+    (Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length,
+     Utils.serialize(rdd.partitions).length)
+  }
+
+  /**
+   * Serialize and deserialize an object. This is useful to verify the objects
+   * contents after deserialization (e.g., the contents of an RDD split after
+   * it is sent to a slave along with a task)
+   */
+  def serializeDeserialize[T](obj: T): T = {
+    val bytes = Utils.serialize(obj)
+    Utils.deserialize[T](bytes)
+  }
+}
+
+
+object CheckpointSuite {
+  // This is a custom cogroup function that does not use mapValues like
+  // the PairRDDFunctions.cogroup()
+  def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = {
+    //println("First = " + first + ", second = " + second)
+    new CoGroupedRDD[K](
+      Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]),
+      part
+    ).asInstanceOf[RDD[(K, Seq[Seq[V]])]]
+  }
+
+}
diff --git a/core/src/test/scala/spark/ClosureCleanerSuite.scala b/core/src/test/scala/spark/ClosureCleanerSuite.scala
index 7c0334d95788bdea7edec97c23172fe524a2a887..b2d0dd4627e0613ae1ce69d92943ddf3acd83174 100644
--- a/core/src/test/scala/spark/ClosureCleanerSuite.scala
+++ b/core/src/test/scala/spark/ClosureCleanerSuite.scala
@@ -3,6 +3,7 @@ package spark
 import java.io.NotSerializableException
 
 import org.scalatest.FunSuite
+import spark.LocalSparkContext._
 import SparkContext._
 
 class ClosureCleanerSuite extends FunSuite {
@@ -43,11 +44,10 @@ object TestObject {
   def run(): Int = {
     var nonSer = new NonSerializable
     var x = 5
-    val sc = new SparkContext("local", "test")
-    val nums = sc.parallelize(Array(1, 2, 3, 4))
-    val answer = nums.map(_ + x).reduce(_ + _)
-    sc.stop()
-    return answer
+    return withSpark(new SparkContext("local", "test")) { sc =>
+      val nums = sc.parallelize(Array(1, 2, 3, 4))
+      nums.map(_ + x).reduce(_ + _)
+    }
   }
 }
 
@@ -58,11 +58,10 @@ class TestClass extends Serializable {
 
   def run(): Int = {
     var nonSer = new NonSerializable
-    val sc = new SparkContext("local", "test")
-    val nums = sc.parallelize(Array(1, 2, 3, 4))
-    val answer = nums.map(_ + getX).reduce(_ + _)
-    sc.stop()
-    return answer
+    return withSpark(new SparkContext("local", "test")) { sc =>
+      val nums = sc.parallelize(Array(1, 2, 3, 4))
+      nums.map(_ + getX).reduce(_ + _)
+    }
   }
 }
 
@@ -71,11 +70,10 @@ class TestClassWithoutDefaultConstructor(x: Int) extends Serializable {
 
   def run(): Int = {
     var nonSer = new NonSerializable
-    val sc = new SparkContext("local", "test")
-    val nums = sc.parallelize(Array(1, 2, 3, 4))
-    val answer = nums.map(_ + getX).reduce(_ + _)
-    sc.stop()
-    return answer
+    return withSpark(new SparkContext("local", "test")) { sc =>
+      val nums = sc.parallelize(Array(1, 2, 3, 4))
+      nums.map(_ + getX).reduce(_ + _)
+    }
   }
 }
 
@@ -87,11 +85,10 @@ class TestClassWithoutFieldAccess {
   def run(): Int = {
     var nonSer2 = new NonSerializable
     var x = 5
-    val sc = new SparkContext("local", "test")
-    val nums = sc.parallelize(Array(1, 2, 3, 4))
-    val answer = nums.map(_ + x).reduce(_ + _)
-    sc.stop()
-    return answer
+    return withSpark(new SparkContext("local", "test")) { sc =>
+      val nums = sc.parallelize(Array(1, 2, 3, 4))
+      nums.map(_ + x).reduce(_ + _)
+    }
   }
 }
 
@@ -100,16 +97,16 @@ object TestObjectWithNesting {
   def run(): Int = {
     var nonSer = new NonSerializable
     var answer = 0
-    val sc = new SparkContext("local", "test")
-    val nums = sc.parallelize(Array(1, 2, 3, 4))
-    var y = 1
-    for (i <- 1 to 4) {
-      var nonSer2 = new NonSerializable
-      var x = i
-      answer += nums.map(_ + x + y).reduce(_ + _)
+    return withSpark(new SparkContext("local", "test")) { sc =>
+      val nums = sc.parallelize(Array(1, 2, 3, 4))
+      var y = 1
+      for (i <- 1 to 4) {
+        var nonSer2 = new NonSerializable
+        var x = i
+        answer += nums.map(_ + x + y).reduce(_ + _)
+      }
+      answer
     }
-    sc.stop()
-    return answer
   }
 }
 
@@ -119,14 +116,14 @@ class TestClassWithNesting(val y: Int) extends Serializable {
   def run(): Int = {
     var nonSer = new NonSerializable
     var answer = 0
-    val sc = new SparkContext("local", "test")
-    val nums = sc.parallelize(Array(1, 2, 3, 4))
-    for (i <- 1 to 4) {
-      var nonSer2 = new NonSerializable
-      var x = i
-      answer += nums.map(_ + x + getY).reduce(_ + _)
+    return withSpark(new SparkContext("local", "test")) { sc =>
+      val nums = sc.parallelize(Array(1, 2, 3, 4))
+      for (i <- 1 to 4) {
+        var nonSer2 = new NonSerializable
+        var x = i
+        answer += nums.map(_ + x + getY).reduce(_ + _)
+      }
+      answer
     }
-    sc.stop()
-    return answer
   }
 }
diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala
index cacc2796b6fb1f69990f6bfd19690d80e290a126..4104b33c8b6815ddebbe50ea595e633fb0cba46e 100644
--- a/core/src/test/scala/spark/DistributedSuite.scala
+++ b/core/src/test/scala/spark/DistributedSuite.scala
@@ -1,5 +1,6 @@
 package spark
 
+import network.ConnectionManagerId
 import org.scalatest.FunSuite
 import org.scalatest.BeforeAndAfter
 import org.scalatest.matchers.ShouldMatchers
@@ -13,43 +14,30 @@ import com.google.common.io.Files
 import scala.collection.mutable.ArrayBuffer
 
 import SparkContext._
-import storage.StorageLevel
+import storage.{GetBlock, BlockManagerWorker, StorageLevel}
 
-class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
+class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter with LocalSparkContext {
 
   val clusterUrl = "local-cluster[2,1,512]"
 
-  @transient var sc: SparkContext = _
-
   after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
     System.clearProperty("spark.reducer.maxMbInFlight")
     System.clearProperty("spark.storage.memoryFraction")
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
   }
 
   test("local-cluster format") {
     sc = new SparkContext("local-cluster[2,1,512]", "test")
     assert(sc.parallelize(1 to 2, 2).count() == 2)
-    sc.stop()
-    System.clearProperty("spark.master.port")
+    resetSparkContext()
     sc = new SparkContext("local-cluster[2 , 1 , 512]", "test")
     assert(sc.parallelize(1 to 2, 2).count() == 2)
-    sc.stop()
-    System.clearProperty("spark.master.port")
+    resetSparkContext()
     sc = new SparkContext("local-cluster[2, 1, 512]", "test")
     assert(sc.parallelize(1 to 2, 2).count() == 2)
-    sc.stop()
-    System.clearProperty("spark.master.port")
+    resetSparkContext()
     sc = new SparkContext("local-cluster[ 2, 1, 512 ]", "test")
     assert(sc.parallelize(1 to 2, 2).count() == 2)
-    sc.stop()
-    System.clearProperty("spark.master.port")
-    sc = null
+    resetSparkContext()
   }
 
   test("simple groupByKey") {
@@ -153,9 +141,22 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
   test("caching in memory and disk, serialized, replicated") {
     sc = new SparkContext(clusterUrl, "test")
     val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2)
+
     assert(data.count() === 1000)
     assert(data.count() === 1000)
     assert(data.count() === 1000)
+
+    // Get all the locations of the first partition and try to fetch the partitions
+    // from those locations.
+    val blockIds = data.partitions.indices.map(index => "rdd_%d_%d".format(data.id, index)).toArray
+    val blockId = blockIds(0)
+    val blockManager = SparkEnv.get.blockManager
+    blockManager.master.getLocations(blockId).foreach(id => {
+      val bytes = BlockManagerWorker.syncGetBlock(
+        GetBlock(blockId), ConnectionManagerId(id.ip, id.port))
+      val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList
+      assert(deserialized === (1 to 100).toList)
+    })
   }
 
   test("compute without caching when no partitions fit in memory") {
@@ -188,4 +189,94 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
     val values = sc.parallelize(1 to 2, 2).map(x => System.getenv("TEST_VAR")).collect()
     assert(values.toSeq === Seq("TEST_VALUE", "TEST_VALUE"))
   }
+
+  test("recover from node failures") {
+    import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity}
+    DistributedSuite.amMaster = true
+    sc = new SparkContext(clusterUrl, "test")
+    val data = sc.parallelize(Seq(true, true), 2)
+    assert(data.count === 2) // force executors to start
+    val masterId = SparkEnv.get.blockManager.blockManagerId
+    assert(data.map(markNodeIfIdentity).collect.size === 2)
+    assert(data.map(failOnMarkedIdentity).collect.size === 2)
+  }
+
+  test("recover from repeated node failures during shuffle-map") {
+    import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity}
+    DistributedSuite.amMaster = true
+    sc = new SparkContext(clusterUrl, "test")
+    for (i <- 1 to 3) {
+      val data = sc.parallelize(Seq(true, false), 2)
+      assert(data.count === 2)
+      assert(data.map(markNodeIfIdentity).collect.size === 2)
+      assert(data.map(failOnMarkedIdentity).map(x => x -> x).groupByKey.count === 2)
+    }
+  }
+
+  test("recover from repeated node failures during shuffle-reduce") {
+    import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity}
+    DistributedSuite.amMaster = true
+    sc = new SparkContext(clusterUrl, "test")
+    for (i <- 1 to 3) {
+      val data = sc.parallelize(Seq(true, true), 2)
+      assert(data.count === 2)
+      assert(data.map(markNodeIfIdentity).collect.size === 2)
+      // This relies on mergeCombiners being used to perform the actual reduce for this
+      // test to actually be testing what it claims.
+      val grouped = data.map(x => x -> x).combineByKey(
+                      x => x,
+                      (x: Boolean, y: Boolean) => x,
+                      (x: Boolean, y: Boolean) => failOnMarkedIdentity(x)
+                    )
+      assert(grouped.collect.size === 1)
+    }
+  }
+
+  test("recover from node failures with replication") {
+    import DistributedSuite.{markNodeIfIdentity, failOnMarkedIdentity}
+    DistributedSuite.amMaster = true
+    // Using more than two nodes so we don't have a symmetric communication pattern and might
+    // cache a partially correct list of peers.
+    sc = new SparkContext("local-cluster[3,1,512]", "test")
+    for (i <- 1 to 3) {
+      val data = sc.parallelize(Seq(true, false, false, false), 4)
+      data.persist(StorageLevel.MEMORY_ONLY_2)
+
+      assert(data.count === 4)
+      assert(data.map(markNodeIfIdentity).collect.size === 4)
+      assert(data.map(failOnMarkedIdentity).collect.size === 4)
+
+      // Create a new replicated RDD to make sure that cached peer information doesn't cause
+      // problems.
+      val data2 = sc.parallelize(Seq(true, true), 2).persist(StorageLevel.MEMORY_ONLY_2)
+      assert(data2.count === 2)
+    }
+  }
+}
+
+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
+
+  // Act like an identity function, but if the argument is true, set mark to true.
+  def markNodeIfIdentity(item: Boolean): Boolean = {
+    if (item) {
+      assert(!amMaster)
+      mark = true
+    }
+    item
+  }
+
+  // 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) { 
+      System.exit(42)
+    } 
+    item
+  } 
 }
diff --git a/core/src/test/scala/spark/DriverSuite.scala b/core/src/test/scala/spark/DriverSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5e84b3a66a1a9a885f7bcea95b4d60eedab4c9e1
--- /dev/null
+++ b/core/src/test/scala/spark/DriverSuite.scala
@@ -0,0 +1,33 @@
+package spark
+
+import java.io.File
+
+import org.scalatest.FunSuite
+import org.scalatest.concurrent.Timeouts
+import org.scalatest.prop.TableDrivenPropertyChecks._
+import org.scalatest.time.SpanSugar._
+
+class DriverSuite extends FunSuite with Timeouts {
+  test("driver should exit after finishing") {
+    assert(System.getenv("SPARK_HOME") != null)
+    // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing"
+    val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
+    forAll(masters) { (master: String) =>
+      failAfter(30 seconds) {
+        Utils.execute(Seq("./run", "spark.DriverWithoutCleanup", master),
+          new File(System.getenv("SPARK_HOME")))
+      }
+    }
+  }
+}
+
+/**
+ * Program that creates a Spark driver but doesn't call SparkContext.stop() or
+ * Sys.exit() after finishing.
+ */
+object DriverWithoutCleanup {
+  def main(args: Array[String]) {
+    val sc = new SparkContext(args(0), "DriverWithoutCleanup")
+    sc.parallelize(1 to 100, 4).count()
+  }
+}
diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala
index a3454f25f6f811ba63ada0460bc1ec709c6fa1b6..8c1445a4656af643d42c2ede53ad6dfadee01b57 100644
--- a/core/src/test/scala/spark/FailureSuite.scala
+++ b/core/src/test/scala/spark/FailureSuite.scala
@@ -1,7 +1,6 @@
 package spark
 
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 import org.scalatest.prop.Checkers
 
 import scala.collection.mutable.ArrayBuffer
@@ -23,18 +22,7 @@ object FailureSuiteState {
   }
 }
 
-class FailureSuite extends FunSuite with BeforeAndAfter {
-  
-  var sc: SparkContext = _
-    
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class FailureSuite extends FunSuite with LocalSparkContext {
   
   // Run a 3-task map job in which task 1 deterministically fails once, and check
   // whether the job completes successfully and we ran 4 tasks in total.
diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala
index b4283d960488b35f21e392f8cc963655ab089221..f1a35bced3ad7df19f848709811feeed85feca7f 100644
--- a/core/src/test/scala/spark/FileServerSuite.scala
+++ b/core/src/test/scala/spark/FileServerSuite.scala
@@ -2,17 +2,16 @@ package spark
 
 import com.google.common.io.Files
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 import java.io.{File, PrintWriter, FileReader, BufferedReader}
 import SparkContext._
 
-class FileServerSuite extends FunSuite with BeforeAndAfter {
+class FileServerSuite extends FunSuite with LocalSparkContext {
 
-  @transient var sc: SparkContext = _
-  @transient var tmpFile : File = _
-  @transient var testJarFile : File = _
+  @transient var tmpFile: File = _
+  @transient var testJarFile: File = _
 
-  before {
+  override def beforeEach() {
+    super.beforeEach()
     // Create a sample text file
     val tmpdir = new File(Files.createTempDir(), "test")
     tmpdir.mkdir()
@@ -22,17 +21,12 @@ class FileServerSuite extends FunSuite with BeforeAndAfter {
     pw.close()
   }
 
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
+  override def afterEach() {
+    super.afterEach()
     // Clean up downloaded file
     if (tmpFile.exists) {
       tmpFile.delete()
     }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
   }
 
   test("Distributing files locally") {
@@ -40,7 +34,8 @@ class FileServerSuite extends FunSuite with BeforeAndAfter {
     sc.addFile(tmpFile.toString)
     val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
     val result = sc.parallelize(testData).reduceByKey {
-      val in  = new BufferedReader(new FileReader("FileServerSuite.txt"))
+      val path = SparkFiles.get("FileServerSuite.txt")
+      val in  = new BufferedReader(new FileReader(path))
       val fileVal = in.readLine().toInt
       in.close()
       _ * fileVal + _ * fileVal
@@ -54,7 +49,8 @@ class FileServerSuite extends FunSuite with BeforeAndAfter {
     sc.addFile((new File(tmpFile.toString)).toURL.toString)
     val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
     val result = sc.parallelize(testData).reduceByKey {
-      val in  = new BufferedReader(new FileReader("FileServerSuite.txt"))
+      val path = SparkFiles.get("FileServerSuite.txt")
+      val in  = new BufferedReader(new FileReader(path))
       val fileVal = in.readLine().toInt
       in.close()
       _ * fileVal + _ * fileVal
@@ -83,7 +79,8 @@ class FileServerSuite extends FunSuite with BeforeAndAfter {
     sc.addFile(tmpFile.toString)
     val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
     val result = sc.parallelize(testData).reduceByKey {
-      val in  = new BufferedReader(new FileReader("FileServerSuite.txt"))
+      val path = SparkFiles.get("FileServerSuite.txt")
+      val in  = new BufferedReader(new FileReader(path))
       val fileVal = in.readLine().toInt
       in.close()
       _ * fileVal + _ * fileVal
diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala
index 554bea53a9181a1fa6c8dad13f9eec94a6be8dc9..91b48c745659b2f1a7fa3136da036c4acd731ce9 100644
--- a/core/src/test/scala/spark/FileSuite.scala
+++ b/core/src/test/scala/spark/FileSuite.scala
@@ -6,24 +6,12 @@ import scala.io.Source
 
 import com.google.common.io.Files
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 import org.apache.hadoop.io._
 
 import SparkContext._
 
-class FileSuite extends FunSuite with BeforeAndAfter {
-  
-  var sc: SparkContext = _
-  
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
-  
+class FileSuite extends FunSuite with LocalSparkContext {
+
   test("text files") {
     sc = new SparkContext("local", "test")
     val tempDir = Files.createTempDir()
diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java
index c61913fc82797e1ede6bd46f47a83ba50af73e7c..d3dcd3bbebe7e472a69516e1101d44e2219753d3 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/spark/JavaAPISuite.java
@@ -46,7 +46,7 @@ public class JavaAPISuite implements Serializable {
     sc.stop();
     sc = null;
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port");
+    System.clearProperty("spark.driver.port");
   }
 
   static class ReverseIntComparator implements Comparator<Integer>, Serializable {
@@ -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(
@@ -355,6 +377,34 @@ public class JavaAPISuite implements Serializable {
     Assert.assertEquals(11, pairs.count());
   }
 
+  @Test
+  public void mapsFromPairsToPairs() {
+      List<Tuple2<Integer, String>> pairs = Arrays.asList(
+              new Tuple2<Integer, String>(1, "a"),
+              new Tuple2<Integer, String>(2, "aa"),
+              new Tuple2<Integer, String>(3, "aaa")
+      );
+      JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs);
+
+      // Regression test for SPARK-668:
+      JavaPairRDD<String, Integer> swapped = pairRDD.flatMap(
+          new PairFlatMapFunction<Tuple2<Integer, String>, String, Integer>() {
+          @Override
+          public Iterable<Tuple2<String, Integer>> call(Tuple2<Integer, String> item) throws Exception {
+              return Collections.singletonList(item.swap());
+          }
+      });
+      swapped.collect();
+
+      // There was never a bug here, but it's worth testing:
+      pairRDD.map(new PairFunction<Tuple2<Integer, String>, String, Integer>() {
+          @Override
+          public Tuple2<String, Integer> call(Tuple2<Integer, String> item) throws Exception {
+              return item.swap();
+          }
+      }).collect();
+  }
+
   @Test
   public void mapPartitions() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2);
@@ -395,7 +445,7 @@ public class JavaAPISuite implements Serializable {
   @Test
   public void iterator() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
-    TaskContext context = new TaskContext(0, 0, 0);
+    TaskContext context = new TaskContext(0, 0, 0, null);
     Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue());
   }
 
@@ -586,7 +636,7 @@ public class JavaAPISuite implements Serializable {
   public void accumulators() {
     JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
 
-    final Accumulator<Integer> intAccum = sc.accumulator(10);
+    final Accumulator<Integer> intAccum = sc.intAccumulator(10);
     rdd.foreach(new VoidFunction<Integer>() {
       public void call(Integer x) {
         intAccum.add(x);
@@ -594,7 +644,7 @@ public class JavaAPISuite implements Serializable {
     });
     Assert.assertEquals((Integer) 25, intAccum.value());
 
-    final Accumulator<Double> doubleAccum = sc.accumulator(10.0);
+    final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0);
     rdd.foreach(new VoidFunction<Integer>() {
       public void call(Integer x) {
         doubleAccum.add((double) x);
@@ -641,4 +691,55 @@ public class JavaAPISuite implements Serializable {
     Assert.assertEquals(new Tuple2<String, Integer>("1", 1), s.get(0));
     Assert.assertEquals(new Tuple2<String, Integer>("2", 2), s.get(1));
   }
+
+  @Test
+  public void checkpointAndComputation() {
+    File tempDir = Files.createTempDir();
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
+    Assert.assertEquals(false, rdd.isCheckpointed());
+    rdd.checkpoint();
+    rdd.count(); // Forces the DAG to cause a checkpoint
+    Assert.assertEquals(true, rdd.isCheckpointed());
+    Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect());
+  }
+
+  @Test
+  public void checkpointAndRestore() {
+    File tempDir = Files.createTempDir();
+    JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
+    sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
+    Assert.assertEquals(false, rdd.isCheckpointed());
+    rdd.checkpoint();
+    rdd.count(); // Forces the DAG to cause a checkpoint
+    Assert.assertEquals(true, rdd.isCheckpointed());
+
+    Assert.assertTrue(rdd.getCheckpointFile().isPresent());
+    JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get());
+    Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect());
+  }
+
+  @Test
+  public void mapOnPairRDD() {
+    JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1,2,3,4));
+    JavaPairRDD<Integer, Integer> rdd2 = rdd1.map(new PairFunction<Integer, Integer, Integer>() {
+      @Override
+      public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+        return new Tuple2<Integer, Integer>(i, i % 2);
+      }
+    });
+    JavaPairRDD<Integer, Integer> rdd3 = rdd2.map(
+        new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() {
+      @Override
+      public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> in) throws Exception {
+        return new Tuple2<Integer, Integer>(in._2(), in._1());
+      }
+    });
+    Assert.assertEquals(Arrays.asList(
+        new Tuple2<Integer, Integer>(1, 1),
+        new Tuple2<Integer, Integer>(0, 2),
+        new Tuple2<Integer, Integer>(1, 3),
+        new Tuple2<Integer, Integer>(0, 4)), rdd3.collect());
+
+  }
 }
diff --git a/core/src/test/scala/spark/KryoSerializerSuite.scala b/core/src/test/scala/spark/KryoSerializerSuite.scala
index 06d446ea246724675ab79cc3b33deb2952606fb7..327e2ff848a0421eeab7c255966613971b8d56e7 100644
--- a/core/src/test/scala/spark/KryoSerializerSuite.scala
+++ b/core/src/test/scala/spark/KryoSerializerSuite.scala
@@ -82,6 +82,7 @@ class KryoSerializerSuite extends FunSuite {
     check(mutable.HashMap(1 -> "one", 2 -> "two"))
     check(mutable.HashMap("one" -> 1, "two" -> 2))
     check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4))))
+    check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three")))
   }
 
   test("custom registrator") {
diff --git a/core/src/test/scala/spark/LocalSparkContext.scala b/core/src/test/scala/spark/LocalSparkContext.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ff00dd05dd7875e78f426b4e4f9b6bec20cfd079
--- /dev/null
+++ b/core/src/test/scala/spark/LocalSparkContext.scala
@@ -0,0 +1,41 @@
+package spark
+
+import org.scalatest.Suite
+import org.scalatest.BeforeAndAfterEach
+
+/** Manages a local `sc` {@link SparkContext} variable, correctly stopping it after each test. */
+trait LocalSparkContext extends BeforeAndAfterEach { self: Suite =>
+
+  @transient var sc: SparkContext = _
+
+  override def afterEach() {
+    resetSparkContext()
+    super.afterEach()
+  }
+
+  def resetSparkContext() = {
+    if (sc != null) {
+      LocalSparkContext.stop(sc)
+      sc = null
+    }
+  }
+
+}
+
+object LocalSparkContext {
+  def stop(sc: SparkContext) {
+    sc.stop()
+    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+    System.clearProperty("spark.driver.port")
+  }
+
+  /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */
+  def withSpark[T](sc: SparkContext)(f: SparkContext => T) = {
+    try {
+      f(sc)
+    } finally {
+      stop(sc)
+    }
+  }
+
+}
\ 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 d3dd3a8fa4930cdc5dcf2cd8b656d5ce03cba272..3abc584b6a177e0159ddecea2dc7e85ac87eb2cc 100644
--- a/core/src/test/scala/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
@@ -1,17 +1,13 @@
 package spark
 
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 
 import akka.actor._
 import spark.scheduler.MapStatus
 import spark.storage.BlockManagerId
 import spark.util.AkkaUtils
 
-class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter {
-  after {
-    System.clearProperty("spark.master.port")
-  }
+class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
  
   test("compressSize") {
     assert(MapOutputTracker.compressSize(0L) === 0)
@@ -35,58 +31,65 @@ class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter {
 
   test("master start and stop") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTracker(actorSystem, true)
+    val tracker = new MapOutputTracker()
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))
     tracker.stop()
   }
 
   test("master register and fetch") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTracker(actorSystem, true)
+    val tracker = new MapOutputTracker()
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))
     tracker.registerShuffle(10, 2)
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val compressedSize10000 = MapOutputTracker.compressSize(10000L)
     val size1000 = MapOutputTracker.decompressSize(compressedSize1000)
     val size10000 = MapOutputTracker.decompressSize(compressedSize10000)
-    tracker.registerMapOutput(10, 0, new MapStatus(new BlockManagerId("hostA", 1000),
+    tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000),
         Array(compressedSize1000, compressedSize10000)))
-    tracker.registerMapOutput(10, 1, new MapStatus(new BlockManagerId("hostB", 1000),
+    tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000),
         Array(compressedSize10000, compressedSize1000)))
     val statuses = tracker.getServerStatuses(10, 0)
-    assert(statuses.toSeq === Seq((new BlockManagerId("hostA", 1000), size1000),
-                                  (new BlockManagerId("hostB", 1000), size10000)))
+    assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000),
+                                  (BlockManagerId("b", "hostB", 1000), size10000)))
     tracker.stop()
   }
 
   test("master register and unregister and fetch") {
     val actorSystem = ActorSystem("test")
-    val tracker = new MapOutputTracker(actorSystem, true)
+    val tracker = new MapOutputTracker()
+    tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))
     tracker.registerShuffle(10, 2)
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val compressedSize10000 = MapOutputTracker.compressSize(10000L)
     val size1000 = MapOutputTracker.decompressSize(compressedSize1000)
     val size10000 = MapOutputTracker.decompressSize(compressedSize10000)
-    tracker.registerMapOutput(10, 0, new MapStatus(new BlockManagerId("hostA", 1000),
+    tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000),
         Array(compressedSize1000, compressedSize1000, compressedSize1000)))
-    tracker.registerMapOutput(10, 1, new MapStatus(new BlockManagerId("hostB", 1000),
+    tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000),
         Array(compressedSize10000, compressedSize1000, compressedSize1000)))
 
     // As if we had two simulatenous fetch failures
-    tracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000))
-    tracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000))
+    tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000))
+    tracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000))
 
-    // The remaining reduce task might try to grab the output dispite the shuffle failure;
+    // The remaining reduce task might try to grab the output despite the shuffle failure;
     // this should cause it to fail, and the scheduler will ignore the failure due to the
     // stage already being aborted.
     intercept[FetchFailedException] { tracker.getServerStatuses(10, 1) }
   }
 
   test("remote fetch") {
-    System.clearProperty("spark.master.host")
-    val (actorSystem, boundPort) =
-      AkkaUtils.createActorSystem("test", "localhost", 0)
-    System.setProperty("spark.master.port", boundPort.toString)
-    val masterTracker = new MapOutputTracker(actorSystem, true)
-    val slaveTracker = new MapOutputTracker(actorSystem, false)
+    val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", "localhost", 0)
+    val masterTracker = new MapOutputTracker()
+    masterTracker.trackerActor = actorSystem.actorOf(
+        Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker")
+    
+    val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", "localhost", 0)
+    val slaveTracker = new MapOutputTracker()
+    slaveTracker.trackerActor = slaveSystem.actorFor(
+        "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker")
+    
     masterTracker.registerShuffle(10, 1)
     masterTracker.incrementGeneration()
     slaveTracker.updateGeneration(masterTracker.getGeneration)
@@ -95,13 +98,13 @@ class MapOutputTrackerSuite extends FunSuite with BeforeAndAfter {
     val compressedSize1000 = MapOutputTracker.compressSize(1000L)
     val size1000 = MapOutputTracker.decompressSize(compressedSize1000)
     masterTracker.registerMapOutput(10, 0, new MapStatus(
-      new BlockManagerId("hostA", 1000), Array(compressedSize1000)))
+      BlockManagerId("a", "hostA", 1000), Array(compressedSize1000)))
     masterTracker.incrementGeneration()
     slaveTracker.updateGeneration(masterTracker.getGeneration)
     assert(slaveTracker.getServerStatuses(10, 0).toSeq ===
-           Seq((new BlockManagerId("hostA", 1000), size1000)))
+           Seq((BlockManagerId("a", "hostA", 1000), size1000)))
 
-    masterTracker.unregisterMapOutput(10, 0, new BlockManagerId("hostA", 1000))
+    masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000))
     masterTracker.incrementGeneration()
     slaveTracker.updateGeneration(masterTracker.getGeneration)
     intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) }
diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala
index eb3c8f238f1879f318ba4523f61a19a37b2882c2..60db759c25f3b424280e869df5319b21c2c3e7e6 100644
--- a/core/src/test/scala/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/spark/PartitioningSuite.scala
@@ -1,25 +1,12 @@
 package spark
 
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 
 import scala.collection.mutable.ArrayBuffer
 
 import SparkContext._
 
-class PartitioningSuite extends FunSuite with BeforeAndAfter {
-  
-  var sc: SparkContext = _
-  
-  after {
-    if(sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
-  
+class PartitioningSuite extends FunSuite with LocalSparkContext {
   
   test("HashPartitioner equality") {
     val p2 = new HashPartitioner(2)
@@ -97,10 +84,10 @@ class PartitioningSuite extends FunSuite with BeforeAndAfter {
     assert(grouped4.groupByKey(3).partitioner !=  grouped4.partitioner)
     assert(grouped4.groupByKey(4).partitioner === grouped4.partitioner)
 
-    assert(grouped2.join(grouped4).partitioner === grouped2.partitioner)
-    assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped2.partitioner)
-    assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped2.partitioner)
-    assert(grouped2.cogroup(grouped4).partitioner === grouped2.partitioner)
+    assert(grouped2.join(grouped4).partitioner === grouped4.partitioner)
+    assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped4.partitioner)
+    assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped4.partitioner)
+    assert(grouped2.cogroup(grouped4).partitioner === grouped4.partitioner)
 
     assert(grouped2.join(reduced2).partitioner === grouped2.partitioner)
     assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner)
diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala
index 9b84b2922793352091ef5a6c91913f462f38ca48..a6344edf8f1614f835cd5fa333baa73827d9e6ba 100644
--- a/core/src/test/scala/spark/PipedRDDSuite.scala
+++ b/core/src/test/scala/spark/PipedRDDSuite.scala
@@ -1,21 +1,9 @@
 package spark
 
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 import SparkContext._
 
-class PipedRDDSuite extends FunSuite with BeforeAndAfter {
-  
-  var sc: SparkContext = _
-  
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class PipedRDDSuite extends FunSuite with LocalSparkContext {
   
   test("basic pipe") {
     sc = new SparkContext("local", "test")
@@ -51,5 +39,3 @@ class PipedRDDSuite extends FunSuite with BeforeAndAfter {
   }
 
 }
-
-
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index d74e9786c3cfcc190d9f713e53c63a174c97226f..7fbdd44340abe5a144f483ee55b5dc9a891b9437 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -2,32 +2,20 @@ package spark
 
 import scala.collection.mutable.HashMap
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
+import spark.SparkContext._
+import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD, ShuffledRDD}
 
-import spark.rdd.CoalescedRDD
-import SparkContext._
-
-class RDDSuite extends FunSuite with BeforeAndAfter {
-
-  var sc: SparkContext = _
-
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class RDDSuite extends FunSuite with LocalSparkContext {
 
   test("basic operations") {
     sc = new SparkContext("local", "test")
     val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
     assert(nums.collect().toList === List(1, 2, 3, 4))
     val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2)
-    assert(dups.distinct.count === 4)
-    assert(dups.distinct().collect === dups.distinct.collect)
-    assert(dups.distinct(2).collect === dups.distinct.collect)
+    assert(dups.distinct().count() === 4)
+    assert(dups.distinct.count === 4)  // Can distinct and count be called without parentheses?
+    assert(dups.distinct.collect === dups.distinct().collect)
+    assert(dups.distinct(2).collect === dups.distinct().collect)
     assert(nums.reduce(_ + _) === 10)
     assert(nums.fold(0)(_ + _) === 10)
     assert(nums.map(_.toString).collect().toList === List("1", "2", "3", "4"))
@@ -44,6 +32,15 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
       case(split, iter) => Iterator((split, iter.reduceLeft(_ + _)))
     }
     assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7)))
+
+    val partitionSumsWithIndex = nums.mapPartitionsWithIndex {
+      case(split, iter) => Iterator((split, iter.reduceLeft(_ + _)))
+    }
+    assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7)))
+
+    intercept[UnsupportedOperationException] {
+      nums.filter(_ > 5).reduce(_ + _)
+    }
   }
 
   test("SparkContext.union") {
@@ -76,10 +73,23 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
     assert(result.toSet === Set(("a", 6), ("b", 2), ("c", 5)))
   }
 
-  test("checkpointing") {
+  test("basic checkpointing") {
+    import java.io.File
+    val checkpointDir = File.createTempFile("temp", "")
+    checkpointDir.delete()
+
     sc = new SparkContext("local", "test")
-    val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).flatMap(x => 1 to x).checkpoint()
-    assert(rdd.collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4))
+    sc.setCheckpointDir(checkpointDir.toString)
+    val parCollection = sc.makeRDD(1 to 4)
+    val flatMappedRDD = parCollection.flatMap(x => 1 to x)
+    flatMappedRDD.checkpoint()
+    assert(flatMappedRDD.dependencies.head.rdd == parCollection)
+    val result = flatMappedRDD.collect()
+    Thread.sleep(1000)
+    assert(flatMappedRDD.dependencies.head.rdd != parCollection)
+    assert(flatMappedRDD.collect() === result)
+
+    checkpointDir.deleteOnExit()
   }
 
   test("basic caching") {
@@ -91,13 +101,13 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
   }
 
   test("caching with failures") {
-    sc = new SparkContext("local", "test") 
-    val onlySplit = new Split { override def index: Int = 0 }
+    sc = new SparkContext("local", "test")
+    val onlySplit = new Partition { override def index: Int = 0 }
     var shouldFail = true
-    val rdd = new RDD[Int](sc) {
-      override def splits: Array[Split] = Array(onlySplit)
-      override val dependencies = List[Dependency[_]]()
-      override def compute(split: Split, context: TaskContext): Iterator[Int] = {
+    val rdd = new RDD[Int](sc, Nil) {
+      override def getPartitions: Array[Partition] = Array(onlySplit)
+      override val getDependencies = List[Dependency[_]]()
+      override def compute(split: Partition, context: TaskContext): Iterator[Int] = {
         if (shouldFail) {
           throw new Exception("injected failure")
         } else {
@@ -113,35 +123,72 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
     assert(rdd.collect().toList === List(1, 2, 3, 4))
   }
 
+  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)
 
-    val coalesced1 = new CoalescedRDD(data, 2)
+    val coalesced1 = data.coalesce(2)
     assert(coalesced1.collect().toList === (1 to 10).toList)
     assert(coalesced1.glom().collect().map(_.toList).toList ===
       List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10)))
 
     // Check that the narrow dependency is also specified correctly
-    assert(coalesced1.dependencies.head.getParents(0).toList === List(0, 1, 2, 3, 4))
-    assert(coalesced1.dependencies.head.getParents(1).toList === List(5, 6, 7, 8, 9))
+    assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(0).toList ===
+      List(0, 1, 2, 3, 4))
+    assert(coalesced1.dependencies.head.asInstanceOf[NarrowDependency[_]].getParents(1).toList ===
+      List(5, 6, 7, 8, 9))
 
-    val coalesced2 = new CoalescedRDD(data, 3)
+    val coalesced2 = data.coalesce(3)
     assert(coalesced2.collect().toList === (1 to 10).toList)
     assert(coalesced2.glom().collect().map(_.toList).toList ===
       List(List(1, 2, 3), List(4, 5, 6), List(7, 8, 9, 10)))
 
-    val coalesced3 = new CoalescedRDD(data, 10)
+    val coalesced3 = data.coalesce(10)
     assert(coalesced3.collect().toList === (1 to 10).toList)
     assert(coalesced3.glom().collect().map(_.toList).toList ===
       (1 to 10).map(x => List(x)).toList)
 
     // If we try to coalesce into more partitions than the original RDD, it should just
     // keep the original number of partitions.
-    val coalesced4 = new CoalescedRDD(data, 20)
+    val coalesced4 = data.coalesce(20)
     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") {
@@ -155,4 +202,75 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
       nums.zip(sc.parallelize(1 to 4, 1)).collect()
     }
   }
+
+  test("partition pruning") {
+    sc = new SparkContext("local", "test")
+    val data = sc.parallelize(1 to 10, 10)
+    // Note that split number starts from 0, so > 8 means only 10th partition left.
+    val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8)
+    assert(prunedRdd.partitions.size === 1)
+    val prunedData = prunedRdd.collect()
+    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/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index bebb8ebe86d65dec4e74701de5c8ee8c908eb6ff..2b2a90defa4e902a8db7fb5ab2bc13da411b5913 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -1,9 +1,9 @@
 package spark
 
 import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashSet
 
 import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
 import org.scalatest.matchers.ShouldMatchers
 import org.scalatest.prop.Checkers
 import org.scalacheck.Arbitrary._
@@ -15,18 +15,7 @@ import com.google.common.io.Files
 import spark.rdd.ShuffledRDD
 import spark.SparkContext._
 
-class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
-
-  var sc: SparkContext = _
-
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
 
   test("groupByKey") {
     sc = new SparkContext("local", "test")
@@ -110,6 +99,28 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
     val sums = pairs.reduceByKey(_+_, 10).collect()
     assert(sums.toSet === Set((1, 7), (2, 1)))
   }
+  
+  test("reduceByKey with partitioner") {
+    sc = new SparkContext("local", "test")
+    val p = new Partitioner() {
+      def numPartitions = 2
+      def getPartition(key: Any) = key.asInstanceOf[Int]
+    }
+    val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p)
+    val sums = pairs.reduceByKey(_+_)
+    assert(sums.collect().toSet === Set((1, 4), (0, 1)))
+    assert(sums.partitioner === Some(p))
+    // count the dependencies to make sure there is only 1 ShuffledRDD
+    val deps = new HashSet[RDD[_]]()
+    def visit(r: RDD[_]) {
+      for (dep <- r.dependencies) {
+        deps += dep.rdd
+        visit(dep.rdd)
+      }
+    }
+    visit(sums)
+    assert(deps.size === 2) // ShuffledRDD, ParallelCollection
+  }
 
   test("join") {
     sc = new SparkContext("local", "test")
@@ -211,7 +222,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
     sc = new SparkContext("local", "test")
     val emptyDir = Files.createTempDir()
     val file = sc.textFile(emptyDir.getAbsolutePath)
-    assert(file.splits.size == 0)
+    assert(file.partitions.size == 0)
     assert(file.collect().toList === Nil)
     // Test that a shuffle on the file works, because this used to be a bug
     assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil)
@@ -223,6 +234,77 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
     assert(rdd.keys.collect().toList === List(1, 2))
     assert(rdd.values.collect().toList === List("a", "b"))
   }
+
+  test("default partitioner uses partition size") {
+    sc = new SparkContext("local", "test")
+    // specify 2000 partitions
+    val a = sc.makeRDD(Array(1, 2, 3, 4), 2000)
+    // do a map, which loses the partitioner
+    val b = a.map(a => (a, (a * 2).toString))
+    // then a group by, and see we didn't revert to 2 partitions
+    val c = b.groupByKey()
+    assert(c.partitions.size === 2000)
+  }
+
+  test("default partitioner uses largest partitioner") {
+    sc = new SparkContext("local", "test")
+    val a = sc.makeRDD(Array((1, "a"), (2, "b")), 2)
+    val b = sc.makeRDD(Array((1, "a"), (2, "b")), 2000)
+    val c = a.join(b)
+    assert(c.partitions.size === 2000)
+  }
+
+  test("subtract") {
+    sc = new SparkContext("local", "test")
+    val a = sc.parallelize(Array(1, 2, 3), 2)
+    val b = sc.parallelize(Array(2, 3, 4), 4)
+    val c = a.subtract(b)
+    assert(c.collect().toSet === Set(1))
+    assert(c.partitions.size === a.partitions.size)
+  }
+
+  test("subtract 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"), (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.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/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala
index 1ad11ff4c3df7b70cb7eb054b9ffdfc888b5c003..495f957e53f251a9597782530f120aa4e055b08f 100644
--- a/core/src/test/scala/spark/SortingSuite.scala
+++ b/core/src/test/scala/spark/SortingSuite.scala
@@ -5,18 +5,7 @@ import org.scalatest.BeforeAndAfter
 import org.scalatest.matchers.ShouldMatchers
 import SparkContext._
 
-class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with Logging {
-  
-  var sc: SparkContext = _
-  
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers with Logging {
   
   test("sortByKey") {
     sc = new SparkContext("local", "test")
@@ -30,7 +19,7 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
     val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
     val pairs = sc.parallelize(pairArr, 2)
     val sorted = pairs.sortByKey()
-    assert(sorted.splits.size === 2)
+    assert(sorted.partitions.size === 2)
     assert(sorted.collect() === pairArr.sortBy(_._1))
   }
 
@@ -40,17 +29,17 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
     val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
     val pairs = sc.parallelize(pairArr, 2)
     val sorted = pairs.sortByKey(true, 1)
-    assert(sorted.splits.size === 1)
+    assert(sorted.partitions.size === 1)
     assert(sorted.collect() === pairArr.sortBy(_._1))
   }
   
-  test("large array with many splits") {
+  test("large array with many partitions") {
     sc = new SparkContext("local", "test")
     val rand = new scala.util.Random()
     val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
     val pairs = sc.parallelize(pairArr, 2)
     val sorted = pairs.sortByKey(true, 20)
-    assert(sorted.splits.size === 20)
+    assert(sorted.partitions.size === 20)
     assert(sorted.collect() === pairArr.sortBy(_._1))
   }
   
@@ -70,7 +59,7 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
     assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1))
   }
   
-  test("sort descending with many splits") {
+  test("sort descending with many partitions") {
     sc = new SparkContext("local", "test")
     val rand = new scala.util.Random()
     val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala
index e9b1837d894040f162802eeab4786efd5f5063ac..ff315b66935dc997631c9accfd8bbf2b8a1d5bb8 100644
--- a/core/src/test/scala/spark/ThreadingSuite.scala
+++ b/core/src/test/scala/spark/ThreadingSuite.scala
@@ -22,19 +22,7 @@ object ThreadingSuiteState {
   }
 }
 
-class ThreadingSuite extends FunSuite with BeforeAndAfter {
-  
-  var sc: SparkContext = _
-  
-  after {
-    if(sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
-  
+class ThreadingSuite extends FunSuite with LocalSparkContext {
   
   test("accessing SparkContext form a different thread") {
     sc = new SparkContext("local", "test")
diff --git a/core/src/test/scala/spark/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
similarity index 83%
rename from core/src/test/scala/spark/ParallelCollectionSplitSuite.scala
rename to core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
index 450c69bd58adb21fa246f2f6e195c9ff7e1a89b5..d27a2538e4489644a514ce677edf1ee4c29682c8 100644
--- a/core/src/test/scala/spark/ParallelCollectionSplitSuite.scala
+++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
 
 import scala.collection.immutable.NumericRange
 
@@ -11,7 +11,7 @@ import org.scalacheck.Prop._
 class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   test("one element per slice") {
     val data = Array(1, 2, 3)
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices(0).mkString(",") === "1")
     assert(slices(1).mkString(",") === "2")
@@ -20,14 +20,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("one slice") {
     val data = Array(1, 2, 3)
-    val slices = ParallelCollection.slice(data, 1)
+    val slices = ParallelCollectionRDD.slice(data, 1)
     assert(slices.size === 1)
     assert(slices(0).mkString(",") === "1,2,3")
   }
   
   test("equal slices") {
     val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9)
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices(0).mkString(",") === "1,2,3")
     assert(slices(1).mkString(",") === "4,5,6")
@@ -36,7 +36,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("non-equal slices") {
     val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices(0).mkString(",") === "1,2,3")
     assert(slices(1).mkString(",") === "4,5,6")
@@ -45,7 +45,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
 
   test("splitting exclusive range") {
     val data = 0 until 100
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices(0).mkString(",") === (0 to 32).mkString(","))
     assert(slices(1).mkString(",") === (33 to 65).mkString(","))
@@ -54,7 +54,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
 
   test("splitting inclusive range") {
     val data = 0 to 100
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices(0).mkString(",") === (0 to 32).mkString(","))
     assert(slices(1).mkString(",") === (33 to 66).mkString(","))
@@ -63,24 +63,24 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("empty data") {
     val data = new Array[Int](0)
-    val slices = ParallelCollection.slice(data, 5)
+    val slices = ParallelCollectionRDD.slice(data, 5)
     assert(slices.size === 5)
     for (slice <- slices) assert(slice.size === 0)
   }
  
   test("zero slices") {
     val data = Array(1, 2, 3)
-    intercept[IllegalArgumentException] { ParallelCollection.slice(data, 0) }
+    intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) }
   }
 
   test("negative number of slices") {
     val data = Array(1, 2, 3)
-    intercept[IllegalArgumentException] { ParallelCollection.slice(data, -5) }
+    intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) }
   }
   
   test("exclusive ranges sliced into ranges") {
     val data = 1 until 100
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices.map(_.size).reduceLeft(_+_) === 99)
     assert(slices.forall(_.isInstanceOf[Range]))
@@ -88,7 +88,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("inclusive ranges sliced into ranges") {
     val data = 1 to 100
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices.map(_.size).reduceLeft(_+_) === 100)
     assert(slices.forall(_.isInstanceOf[Range]))
@@ -97,7 +97,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   test("large ranges don't overflow") {
     val N = 100 * 1000 * 1000
     val data = 0 until N
-    val slices = ParallelCollection.slice(data, 40)
+    val slices = ParallelCollectionRDD.slice(data, 40)
     assert(slices.size === 40)
     for (i <- 0 until 40) {
       assert(slices(i).isInstanceOf[Range])
@@ -117,7 +117,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
       (tuple: (List[Int], Int)) =>
         val d = tuple._1
         val n = tuple._2
-        val slices = ParallelCollection.slice(d, n)
+        val slices = ParallelCollectionRDD.slice(d, n)
         ("n slices"    |: slices.size == n) &&
         ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) &&
         ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1))
@@ -134,7 +134,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
     } yield (a until b by step, n)
     val prop = forAll(gen) {
       case (d: Range, n: Int) =>
-        val slices = ParallelCollection.slice(d, n)
+        val slices = ParallelCollectionRDD.slice(d, n)
         ("n slices"    |: slices.size == n) &&
         ("all ranges"  |: slices.forall(_.isInstanceOf[Range])) &&
         ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) &&
@@ -152,7 +152,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
     } yield (a to b by step, n)
     val prop = forAll(gen) {
       case (d: Range, n: Int) =>
-        val slices = ParallelCollection.slice(d, n)
+        val slices = ParallelCollectionRDD.slice(d, n)
         ("n slices"    |: slices.size == n) &&
         ("all ranges"  |: slices.forall(_.isInstanceOf[Range])) &&
         ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) &&
@@ -163,7 +163,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("exclusive ranges of longs") {
     val data = 1L until 100L
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices.map(_.size).reduceLeft(_+_) === 99)
     assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
@@ -171,7 +171,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("inclusive ranges of longs") {
     val data = 1L to 100L
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices.map(_.size).reduceLeft(_+_) === 100)
     assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
@@ -179,7 +179,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("exclusive ranges of doubles") {
     val data = 1.0 until 100.0 by 1.0
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices.map(_.size).reduceLeft(_+_) === 99)
     assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
@@ -187,7 +187,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
   
   test("inclusive ranges of doubles") {
     val data = 1.0 to 100.0 by 1.0
-    val slices = ParallelCollection.slice(data, 3)
+    val slices = ParallelCollectionRDD.slice(data, 3)
     assert(slices.size === 3)
     assert(slices.map(_.size).reduceLeft(_+_) === 100)
     assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6da58a0f6e5fe3beeb6117382960e3c57ab679e9
--- /dev/null
+++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
@@ -0,0 +1,403 @@
+package spark.scheduler
+
+import scala.collection.mutable.{Map, HashMap}
+
+import org.scalatest.FunSuite
+import org.scalatest.BeforeAndAfter
+
+import spark.LocalSparkContext
+
+import spark.storage.BlockManager
+import spark.storage.BlockManagerId
+import spark.storage.BlockManagerMaster
+import spark.{Dependency, ShuffleDependency, OneToOneDependency}
+import spark.FetchFailedException
+import spark.MapOutputTracker
+import spark.RDD
+import spark.SparkContext
+import spark.SparkException
+import spark.Partition
+import spark.TaskContext
+import spark.TaskEndReason
+
+import spark.{FetchFailed, Success}
+
+/**
+ * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler
+ * rather than spawning an event loop thread as happens in the real code. They use EasyMock
+ * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are
+ * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead
+ * host notifications are sent). In addition, tests may check for side effects on a non-mocked
+ * MapOutputTracker instance.
+ *
+ * Tests primarily consist of running DAGScheduler#processEvent and
+ * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet)
+ * and capturing the resulting TaskSets from the mock TaskScheduler.
+ */
+class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
+
+  /** Set of TaskSets the DAGScheduler has requested executed. */
+  val taskSets = scala.collection.mutable.Buffer[TaskSet]()
+  val taskScheduler = new TaskScheduler() {
+    override def start() = {}
+    override def stop() = {}
+    override def submitTasks(taskSet: TaskSet) = {
+      // normally done by TaskSetManager
+      taskSet.tasks.foreach(_.generation = mapOutputTracker.getGeneration)
+      taskSets += taskSet 
+    }
+    override def setListener(listener: TaskSchedulerListener) = {}
+    override def defaultParallelism() = 2
+  }
+
+  var mapOutputTracker: MapOutputTracker = null
+  var scheduler: DAGScheduler = null
+
+  /**
+   * Set of cache locations to return from our mock BlockManagerMaster.
+   * Keys are (rdd ID, partition ID). Anything not present will return an empty
+   * list of cache locations silently.
+   */
+  val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]]
+  // stub out BlockManagerMaster.getLocations to use our cacheLocations
+  val blockManagerMaster = new BlockManagerMaster(null) {
+      override def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
+        blockIds.map { name =>
+          val pieces = name.split("_")
+          if (pieces(0) == "rdd") {
+            val key = pieces(1).toInt -> pieces(2).toInt
+            cacheLocations.getOrElse(key, Seq())
+          } else {
+            Seq()
+          }
+        }.toSeq
+      }
+      override def removeExecutor(execId: String) {
+        // don't need to propagate to the driver, which we don't have
+      }
+    }
+
+  /** The list of results that DAGScheduler has collected. */
+  val results = new HashMap[Int, Any]()
+  var failure: Exception = _
+  val listener = new JobListener() {
+    override def taskSucceeded(index: Int, result: Any) = results.put(index, result)
+    override def jobFailed(exception: Exception) = { failure = exception }
+  }
+
+  before {
+    sc = new SparkContext("local", "DAGSchedulerSuite")
+    taskSets.clear()
+    cacheLocations.clear()
+    results.clear()
+    mapOutputTracker = new MapOutputTracker()
+    scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, null) {
+      override def runLocally(job: ActiveJob) {
+        // don't bother with the thread while unit testing
+        runLocallyWithinThread(job)
+      }
+    }
+  }
+
+  after {
+    scheduler.stop()
+  }
+
+  /**
+   * Type of RDD we use for testing. Note that we should never call the real RDD compute methods.
+   * This is a pair RDD type so it can always be used in ShuffleDependencies.
+   */
+  type MyRDD = RDD[(Int, Int)]
+
+  /**
+   * Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and
+   * preferredLocations (if any) that are passed to them. They are deliberately not executable
+   * so we can test that DAGScheduler does not try to execute RDDs locally.
+   */
+  private def makeRdd(
+        numPartitions: Int,
+        dependencies: List[Dependency[_]],
+        locations: Seq[Seq[String]] = Nil
+      ): MyRDD = {
+    val maxPartition = numPartitions - 1
+    return new MyRDD(sc, dependencies) {
+      override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] =
+        throw new RuntimeException("should not be reached")
+      override def getPartitions = (0 to maxPartition).map(i => new Partition {
+        override def index = i
+      }).toArray
+      override def getPreferredLocations(split: Partition): Seq[String] =
+        if (locations.isDefinedAt(split.index))
+          locations(split.index)
+        else
+          Nil
+      override def toString: String = "DAGSchedulerSuiteRDD " + id
+    }
+  }
+
+  /**
+   * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting
+   * the scheduler not to exit.
+   *
+   * After processing the event, submit waiting stages as is done on most iterations of the
+   * DAGScheduler event loop.
+   */
+  private def runEvent(event: DAGSchedulerEvent) {
+    assert(!scheduler.processEvent(event))
+    scheduler.submitWaitingStages()
+  }
+
+  /**
+   * When we submit dummy Jobs, this is the compute function we supply. Except in a local test
+   * below, we do not expect this function to ever be executed; instead, we will return results
+   * directly through CompletionEvents.
+   */
+  private val jobComputeFunc = (context: TaskContext, it: Iterator[(_)]) =>
+     it.next.asInstanceOf[Tuple2[_, _]]._1
+
+  /** Send the given CompletionEvent messages for the tasks in the TaskSet. */
+  private def complete(taskSet: TaskSet, results: Seq[(TaskEndReason, Any)]) {
+    assert(taskSet.tasks.size >= results.size)
+    for ((result, i) <- results.zipWithIndex) {
+      if (i < taskSet.tasks.size) {
+        runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null, null))
+      }
+    }
+  }
+     
+  /** Sends the rdd to the scheduler for scheduling. */
+  private def submit(
+      rdd: RDD[_],
+      partitions: Array[Int],
+      func: (TaskContext, Iterator[_]) => _ = jobComputeFunc,
+      allowLocal: Boolean = false,
+      listener: JobListener = listener) {
+    runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener))
+  }
+  
+  /** Sends TaskSetFailed to the scheduler. */
+  private def failed(taskSet: TaskSet, message: String) {
+    runEvent(TaskSetFailed(taskSet, message))
+  }
+
+  test("zero split job") {
+    val rdd = makeRdd(0, Nil)
+    var numResults = 0
+    val fakeListener = new JobListener() {
+      override def taskSucceeded(partition: Int, value: Any) = numResults += 1
+      override def jobFailed(exception: Exception) = throw exception
+    }
+    submit(rdd, Array(), listener = fakeListener)
+    assert(numResults === 0)
+  }
+
+  test("run trivial job") {
+    val rdd = makeRdd(1, Nil)
+    submit(rdd, Array(0))
+    complete(taskSets(0), List((Success, 42)))
+    assert(results === Map(0 -> 42))
+  }
+
+  test("local job") {
+    val rdd = new MyRDD(sc, Nil) {
+      override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] =
+        Array(42 -> 0).iterator
+      override def getPartitions = Array( new Partition { override def index = 0 } )
+      override def getPreferredLocations(split: Partition) = Nil
+      override def toString = "DAGSchedulerSuite Local RDD"
+    }
+    runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener))
+    assert(results === Map(0 -> 42))
+  }
+  
+  test("run trivial job w/ dependency") {
+    val baseRdd = makeRdd(1, Nil)
+    val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd)))
+    submit(finalRdd, Array(0)) 
+    complete(taskSets(0), Seq((Success, 42)))
+    assert(results === Map(0 -> 42))
+  }
+
+  test("cache location preferences w/ dependency") {
+    val baseRdd = makeRdd(1, Nil)
+    val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd)))
+    cacheLocations(baseRdd.id -> 0) =
+      Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))
+    submit(finalRdd, Array(0))
+    val taskSet = taskSets(0)
+    assertLocations(taskSet, Seq(Seq("hostA", "hostB")))
+    complete(taskSet, Seq((Success, 42)))
+    assert(results === Map(0 -> 42))
+  }
+
+  test("trivial job failure") {
+    submit(makeRdd(1, Nil), Array(0))
+    failed(taskSets(0), "some failure")
+    assert(failure.getMessage === "Job failed: some failure")
+  }
+
+  test("run trivial shuffle") {
+    val shuffleMapRdd = makeRdd(2, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
+    val shuffleId = shuffleDep.shuffleId
+    val reduceRdd = makeRdd(1, List(shuffleDep))
+    submit(reduceRdd, Array(0))
+    complete(taskSets(0), Seq(
+        (Success, makeMapStatus("hostA", 1)),
+        (Success, makeMapStatus("hostB", 1))))
+    assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
+           Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
+    complete(taskSets(1), Seq((Success, 42)))
+    assert(results === Map(0 -> 42))
+  }
+  
+  test("run trivial shuffle with fetch failure") {
+    val shuffleMapRdd = makeRdd(2, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
+    val shuffleId = shuffleDep.shuffleId
+    val reduceRdd = makeRdd(2, List(shuffleDep))
+    submit(reduceRdd, Array(0, 1))
+    complete(taskSets(0), Seq(
+        (Success, makeMapStatus("hostA", 1)),
+        (Success, makeMapStatus("hostB", 1))))
+    // the 2nd ResultTask failed
+    complete(taskSets(1), Seq(
+        (Success, 42),
+        (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null)))
+    // this will get called
+    // blockManagerMaster.removeExecutor("exec-hostA")
+    // ask the scheduler to try it again
+    scheduler.resubmitFailedStages()
+    // 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"))
+    complete(taskSets(3), Seq((Success, 43)))
+    assert(results === Map(0 -> 42, 1 -> 43))
+  }
+
+  test("ignore late map task completions") {
+    val shuffleMapRdd = makeRdd(2, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
+    val shuffleId = shuffleDep.shuffleId
+    val reduceRdd = makeRdd(2, List(shuffleDep))
+    submit(reduceRdd, Array(0, 1))
+    // pretend we were told hostA went away
+    val oldGeneration = mapOutputTracker.getGeneration
+    runEvent(ExecutorLost("exec-hostA"))
+    val newGeneration = mapOutputTracker.getGeneration
+    assert(newGeneration > oldGeneration)
+    val noAccum = Map[Long, Any]()
+    val taskSet = taskSets(0)
+    // should be ignored for being too old
+    runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null))
+    // should work because it's a non-failed host
+    runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null, null))
+    // should be ignored for being too old
+    runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null))
+    // should work because it's a new generation
+    taskSet.tasks(1).generation = newGeneration
+    runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null))
+    assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
+           Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA")))
+    complete(taskSets(1), Seq((Success, 42), (Success, 43)))
+    assert(results === Map(0 -> 42, 1 -> 43))
+  }
+
+  test("run trivial shuffle with out-of-band failure and retry") {
+    val shuffleMapRdd = makeRdd(2, Nil)
+    val shuffleDep = new ShuffleDependency(shuffleMapRdd, null)
+    val shuffleId = shuffleDep.shuffleId
+    val reduceRdd = makeRdd(1, List(shuffleDep))
+    submit(reduceRdd, Array(0))
+    // blockManagerMaster.removeExecutor("exec-hostA")
+    // pretend we were told hostA went away
+    runEvent(ExecutorLost("exec-hostA"))
+    // DAGScheduler will immediately resubmit the stage after it appears to have no pending tasks
+    // rather than marking it is as failed and waiting.
+    complete(taskSets(0), Seq(
+        (Success, makeMapStatus("hostA", 1)),
+       (Success, makeMapStatus("hostB", 1))))
+   // have hostC complete the resubmitted task
+   complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1))))
+   assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
+          Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB")))
+   complete(taskSets(2), Seq((Success, 42)))
+   assert(results === Map(0 -> 42))
+ }
+
+ test("recursive shuffle failures") {
+    val shuffleOneRdd = makeRdd(2, Nil)
+    val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
+    val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne))
+    val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null)
+    val finalRdd = makeRdd(1, List(shuffleDepTwo))
+    submit(finalRdd, Array(0))
+    // have the first stage complete normally
+    complete(taskSets(0), Seq(
+        (Success, makeMapStatus("hostA", 2)),
+        (Success, makeMapStatus("hostB", 2))))
+    // have the second stage complete normally
+    complete(taskSets(1), Seq(
+        (Success, makeMapStatus("hostA", 1)),
+        (Success, makeMapStatus("hostC", 1))))
+    // fail the third stage because hostA went down
+    complete(taskSets(2), Seq(
+        (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null)))
+    // TODO assert this:
+    // blockManagerMaster.removeExecutor("exec-hostA")
+    // have DAGScheduler try again
+    scheduler.resubmitFailedStages()
+    complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2))))
+    complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1))))
+    complete(taskSets(5), Seq((Success, 42)))
+    assert(results === Map(0 -> 42))
+  }
+
+  test("cached post-shuffle") {
+    val shuffleOneRdd = makeRdd(2, Nil)
+    val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
+    val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne))
+    val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null)
+    val finalRdd = makeRdd(1, List(shuffleDepTwo))
+    submit(finalRdd, Array(0))
+    cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD"))
+    cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC"))
+    // complete stage 2
+    complete(taskSets(0), Seq(
+        (Success, makeMapStatus("hostA", 2)),
+        (Success, makeMapStatus("hostB", 2))))
+    // complete stage 1
+    complete(taskSets(1), Seq(
+        (Success, makeMapStatus("hostA", 1)),
+        (Success, makeMapStatus("hostB", 1))))
+    // pretend stage 0 failed because hostA went down
+    complete(taskSets(2), Seq(
+        (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null)))
+    // TODO assert this:
+    // blockManagerMaster.removeExecutor("exec-hostA")
+    // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun.
+    scheduler.resubmitFailedStages()
+    assertLocations(taskSets(3), Seq(Seq("hostD")))
+    // allow hostD to recover
+    complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1))))
+    complete(taskSets(4), Seq((Success, 42)))
+    assert(results === Map(0 -> 42))
+  }
+
+  /** Assert that the supplied TaskSet has exactly the given preferredLocations. */
+  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)
+    }
+  }
+
+  private def makeMapStatus(host: String, reduces: Int): MapStatus =
+   new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2))
+
+  private def makeBlockManagerId(host: String): BlockManagerId =
+    BlockManagerId("exec-" + host, host, 12345)
+
+}
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..2f5af10e69c7f2293ee7e071893df883330560b7
--- /dev/null
+++ b/core/src/test/scala/spark/scheduler/SparkListenerSuite.scala
@@ -0,0 +1,86 @@
+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.shuffleReadMillis should be > (0l)
+          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/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
index f9378773406d309ff980cb9981458efba4469697..647bcaf860a37c83354493a805d9aa06aa7f9dba 100644
--- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
@@ -5,28 +5,17 @@ import org.scalatest.BeforeAndAfter
 import spark.TaskContext
 import spark.RDD
 import spark.SparkContext
-import spark.Split
+import spark.Partition
+import spark.LocalSparkContext
 
-class TaskContextSuite extends FunSuite with BeforeAndAfter {
-
-  var sc: SparkContext = _
-
-  after {
-    if (sc != null) {
-      sc.stop()
-      sc = null
-    }
-    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
-    System.clearProperty("spark.master.port")
-  }
+class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
 
   test("Calls executeOnCompleteCallbacks after failure") {
     var completed = false
     sc = new SparkContext("local", "test")
-    val rdd = new RDD[String](sc) {
-      override val splits = Array[Split](StubSplit(0))
-      override val dependencies = List()
-      override def compute(split: Split, context: TaskContext) = {
+    val rdd = new RDD[String](sc, List()) {
+      override def getPartitions = Array[Partition](StubPartition(0))
+      override def compute(split: Partition, context: TaskContext) = {
         context.addOnCompleteCallback(() => completed = true)
         sys.error("failed")
       }
@@ -39,5 +28,5 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter {
     assert(completed === true)
   }
 
-  case class StubSplit(val index: Int) extends Split
-}
\ No newline at end of file
+  case class StubPartition(val index: Int) extends Partition
+}
diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
index 8f86e3170ed0f1b0af2874621ac5221a822dbcea..b8c0f6fb763a9d58251988d12708c7f7776f7454 100644
--- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
@@ -12,6 +12,7 @@ import org.scalatest.concurrent.Timeouts._
 import org.scalatest.matchers.ShouldMatchers._
 import org.scalatest.time.SpanSugar._
 
+import spark.JavaSerializer
 import spark.KryoSerializer
 import spark.SizeEstimator
 import spark.util.ByteBufferInputStream
@@ -31,7 +32,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   before {
     actorSystem = ActorSystem("test")
-    master = new BlockManagerMaster(actorSystem, true, true, "localhost", 7077)
+    master = new BlockManagerMaster(
+      actorSystem.actorOf(Props(new spark.storage.BlockManagerMasterActor(true))))
 
     // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
     oldArch = System.setProperty("os.arch", "amd64")
@@ -69,33 +71,41 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("StorageLevel object caching") {
-    val level1 = new StorageLevel(false, false, false, 3)
-    val level2 = new StorageLevel(false, false, false, 3)
+    val level1 = StorageLevel(false, false, false, 3)
+    val level2 = StorageLevel(false, false, false, 3) // this should return the same object as level1
+    val level3 = StorageLevel(false, false, false, 2) // this should return a different object
+    assert(level2 === level1, "level2 is not same as level1")
+    assert(level2.eq(level1), "level2 is not the same object as level1")
+    assert(level3 != level1, "level3 is same as level1")
     val bytes1 = spark.Utils.serialize(level1)
     val level1_ = spark.Utils.deserialize[StorageLevel](bytes1)
     val bytes2 = spark.Utils.serialize(level2)
     val level2_ = spark.Utils.deserialize[StorageLevel](bytes2)
     assert(level1_ === level1, "Deserialized level1 not same as original level1")
-    assert(level2_ === level2, "Deserialized level2 not same as original level1")
-    assert(level1_ === level2_, "Deserialized level1 not same as deserialized level2")
-    assert(level2_.eq(level1_), "Deserialized level2 not the same object as deserialized level1")
+    assert(level1_.eq(level1), "Deserialized level1 not the same object as original level2")
+    assert(level2_ === level2, "Deserialized level2 not same as original level2")
+    assert(level2_.eq(level1), "Deserialized level2 not the same object as original level1")
   }
 
   test("BlockManagerId object caching") {
-    val id1 = new StorageLevel(false, false, false, 3)
-    val id2 = new StorageLevel(false, false, false, 3)
+    val id1 = BlockManagerId("e1", "XXX", 1)
+    val id2 = BlockManagerId("e1", "XXX", 1) // this should return the same object as id1
+    val id3 = BlockManagerId("e1", "XXX", 2) // this should return a different object
+    assert(id2 === id1, "id2 is not same as id1")
+    assert(id2.eq(id1), "id2 is not the same object as id1")
+    assert(id3 != id1, "id3 is same as id1")
     val bytes1 = spark.Utils.serialize(id1)
-    val id1_ = spark.Utils.deserialize[StorageLevel](bytes1)
+    val id1_ = spark.Utils.deserialize[BlockManagerId](bytes1)
     val bytes2 = spark.Utils.serialize(id2)
-    val id2_ = spark.Utils.deserialize[StorageLevel](bytes2)
-    assert(id1_ === id1, "Deserialized id1 not same as original id1")
-    assert(id2_ === id2, "Deserialized id2 not same as original id1")
-    assert(id1_ === id2_, "Deserialized id1 not same as deserialized id2")
-    assert(id2_.eq(id1_), "Deserialized id2 not the same object as deserialized level1")
+    val id2_ = spark.Utils.deserialize[BlockManagerId](bytes2)
+    assert(id1_ === id1, "Deserialized id1 is not same as original id1")
+    assert(id1_.eq(id1), "Deserialized id1 is not the same object as original id1")
+    assert(id2_ === id2, "Deserialized id2 is not same as original id2")
+    assert(id2_.eq(id1), "Deserialized id2 is not the same object as original id1")
   }
 
   test("master + 1 manager interaction") {
-    store = new BlockManager(actorSystem, master, serializer, 2000)
+    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)
@@ -125,8 +135,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("master + 2 managers interaction") {
-    store = new BlockManager(actorSystem, master, serializer, 2000)
-    store2 = new BlockManager(actorSystem, master, new KryoSerializer, 2000)
+    store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
+    store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000)
 
     val peers = master.getPeers(store.blockManagerId, 1)
     assert(peers.size === 1, "master did not return the other manager as a peer")
@@ -141,7 +151,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("removing block") {
-    store = new BlockManager(actorSystem, master, serializer, 2000)
+    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)
@@ -190,7 +200,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
 
   test("reregistration on heart beat") {
     val heartBeat = PrivateMethod[Unit]('heartBeat)
-    store = new BlockManager(actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
     val a1 = new Array[Byte](400)
 
     store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
@@ -198,7 +208,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
     assert(store.getSingle("a1") != None, "a1 was not in store")
     assert(master.getLocations("a1").size > 0, "master was not told about a1")
 
-    master.notifyADeadHost(store.blockManagerId.ip)
+    master.removeExecutor(store.blockManagerId.executorId)
     assert(master.getLocations("a1").size == 0, "a1 was not removed from master")
 
     store invokePrivate heartBeat()
@@ -206,25 +216,63 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("reregistration on block update") {
-    store = new BlockManager(actorSystem, master, serializer, 2000)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
 
     store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
-
     assert(master.getLocations("a1").size > 0, "master was not told about a1")
 
-    master.notifyADeadHost(store.blockManagerId.ip)
+    master.removeExecutor(store.blockManagerId.executorId)
     assert(master.getLocations("a1").size == 0, "a1 was not removed from master")
 
     store.putSingle("a2", a1, StorageLevel.MEMORY_ONLY)
+    store.waitForAsyncReregister()
 
     assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master")
     assert(master.getLocations("a2").size > 0, "master was not told about a2")
   }
 
+  test("reregistration doesn't dead lock") {
+    val heartBeat = PrivateMethod[Unit]('heartBeat)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+    val a1 = new Array[Byte](400)
+    val a2 = List(new Array[Byte](400))
+
+    // try many times to trigger any deadlocks
+    for (i <- 1 to 100) {
+      master.removeExecutor(store.blockManagerId.executorId)
+      val t1 = new Thread {
+        override def run() {
+          store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, true)
+        }
+      }
+      val t2 = new Thread {
+        override def run() {
+          store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
+        }
+      }
+      val t3 = new Thread {
+        override def run() {
+          store invokePrivate heartBeat()
+        }
+      }
+
+      t1.start()
+      t2.start()
+      t3.start()
+      t1.join()
+      t2.join()
+      t3.join()
+
+      store.dropFromMemory("a1", null)
+      store.dropFromMemory("a2", null)
+      store.waitForAsyncReregister()
+    }
+  }
+
   test("in-memory LRU storage") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -243,7 +291,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU storage with serialization") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -262,14 +310,14 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU for partitions of same RDD") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
     store.putSingle("rdd_0_1", a1, StorageLevel.MEMORY_ONLY)
     store.putSingle("rdd_0_2", a2, StorageLevel.MEMORY_ONLY)
     store.putSingle("rdd_0_3", a3, StorageLevel.MEMORY_ONLY)
-    // Even though we accessed rdd_0_3 last, it should not have replaced partitiosn 1 and 2
+    // Even though we accessed rdd_0_3 last, it should not have replaced partitions 1 and 2
     // from the same RDD
     assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
     assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
@@ -281,7 +329,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU for partitions of multiple RDDs") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
     store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
     store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
@@ -304,7 +352,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("on-disk storage") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -317,7 +365,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -332,7 +380,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage with getLocalBytes") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -347,7 +395,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage with serialization") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -362,7 +410,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("disk and memory storage with serialization and getLocalBytes") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -377,7 +425,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("LRU with mixed storage levels") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val a1 = new Array[Byte](400)
     val a2 = new Array[Byte](400)
     val a3 = new Array[Byte](400)
@@ -402,7 +450,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("in-memory LRU with streams") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val list1 = List(new Array[Byte](200), new Array[Byte](200))
     val list2 = List(new Array[Byte](200), new Array[Byte](200))
     val list3 = List(new Array[Byte](200), new Array[Byte](200))
@@ -426,7 +474,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("LRU with mixed storage levels and streams") {
-    store = new BlockManager(actorSystem, master, serializer, 1200)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
     val list1 = List(new Array[Byte](200), new Array[Byte](200))
     val list2 = List(new Array[Byte](200), new Array[Byte](200))
     val list3 = List(new Array[Byte](200), new Array[Byte](200))
@@ -472,7 +520,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   }
 
   test("overly large block") {
-    store = new BlockManager(actorSystem, master, serializer, 500)
+    store = new BlockManager("<driver>", actorSystem, master, serializer, 500)
     store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
     assert(store.getSingle("a1") === None, "a1 was in store")
     store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK)
@@ -483,49 +531,49 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
   test("block compression") {
     try {
       System.setProperty("spark.shuffle.compress", "true")
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
       store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize("shuffle_0_0_0") <= 100, "shuffle_0_0_0 was not compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.shuffle.compress", "false")
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec2", actorSystem, master, serializer, 2000)
       store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize("shuffle_0_0_0") >= 1000, "shuffle_0_0_0 was compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.broadcast.compress", "true")
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec3", actorSystem, master, serializer, 2000)
       store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize("broadcast_0") <= 100, "broadcast_0 was not compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.broadcast.compress", "false")
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec4", actorSystem, master, serializer, 2000)
       store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize("broadcast_0") >= 1000, "broadcast_0 was compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.rdd.compress", "true")
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec5", actorSystem, master, serializer, 2000)
       store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize("rdd_0_0") <= 100, "rdd_0_0 was not compressed")
       store.stop()
       store = null
 
       System.setProperty("spark.rdd.compress", "false")
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec6", actorSystem, master, serializer, 2000)
       store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
       assert(store.memoryStore.getSize("rdd_0_0") >= 1000, "rdd_0_0 was compressed")
       store.stop()
       store = null
 
       // Check that any other block types are also kept uncompressed
-      store = new BlockManager(actorSystem, master, serializer, 2000)
+      store = new BlockManager("exec7", actorSystem, master, serializer, 2000)
       store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
       assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed")
       store.stop()
@@ -536,4 +584,21 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
       System.clearProperty("spark.rdd.compress")
     }
   }
+
+  test("block store put failure") {
+    // Use Java serializer so we can create an unserializable error.
+    store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer, 1200)
+
+    // The put should fail since a1 is not serializable.
+    class UnserializableClass
+    val a1 = new UnserializableClass
+    intercept[java.io.NotSerializableException] {
+      store.putSingle("a1", a1, StorageLevel.DISK_ONLY)
+    }
+
+    // Make sure get a1 doesn't hang and returns None.
+    failAfter(1 second) {
+      assert(store.getSingle("a1") == None, "a1 should not be in store")
+    }
+  }
 }
diff --git a/core/src/test/scala/spark/util/DistributionSuite.scala b/core/src/test/scala/spark/util/DistributionSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..cc6249b1dda8de4d183e4786f6981bb09d609904
--- /dev/null
+++ b/core/src/test/scala/spark/util/DistributionSuite.scala
@@ -0,0 +1,25 @@
+package spark.util
+
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+
+/**
+ *
+ */
+
+class DistributionSuite extends FunSuite with ShouldMatchers {
+  test("summary") {
+    val d = new Distribution((1 to 100).toArray.map{_.toDouble})
+    val stats = d.statCounter
+    stats.count should be (100)
+    stats.mean should be (50.5)
+    stats.sum should be (50 * 101)
+
+    val quantiles = d.getQuantiles()
+    quantiles(0) should be (1)
+    quantiles(1) should be (26)
+    quantiles(2) should be (51)
+    quantiles(3) should be (76)
+    quantiles(4) should be (100)
+  }
+}
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/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..794063fb6d3a48a6f14c29c2113633add34fd752
--- /dev/null
+++ b/core/src/test/scala/spark/util/RateLimitedOutputStreamSuite.scala
@@ -0,0 +1,23 @@
+package spark.util
+
+import org.scalatest.FunSuite
+import java.io.ByteArrayOutputStream
+import java.util.concurrent.TimeUnit._
+
+class RateLimitedOutputStreamSuite extends FunSuite {
+
+  private def benchmark[U](f: => U): Long = {
+    val start = System.nanoTime
+    f
+    System.nanoTime - start
+  }
+
+  test("write") {
+    val underlying = new ByteArrayOutputStream
+    val data = "X" * 41000
+    val stream = new RateLimitedOutputStream(underlying, 10000)
+    val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) }
+    assert(SECONDS.convert(elapsedNs, NANOSECONDS) == 4)
+    assert(underlying.toString("UTF-8") == data)
+  }
+}
diff --git a/docs/README.md b/docs/README.md
index 887f407f18c666c7fec22e3c8dbcfcf4f5ed9318..c2b3497bb374ecc8b811d895b59032a7ab68fcbb 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -33,4 +33,4 @@ Similarly, you can build just the PySpark epydoc by running `epydoc --config epy
 
 When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc.  The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/).
 
-NOTE: To skip the step of building and copying over the scaladoc when you build the docs, run `SKIP_SCALADOC=1 jekyll`.  Similarly, `SKIP_EPYDOC=1 jekyll` will skip PySpark API doc generation.
+NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`.
diff --git a/docs/_config.yml b/docs/_config.yml
index 2bd2eecc863e40eba17372829dcb6e0ce5d108e9..5c135a024215b0bfc697a938a592f7518f121428 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -3,7 +3,8 @@ markdown: kramdown
 
 # These allow the documentation to be updated with nerw releases
 # of Spark, Scala, and Mesos.
-SPARK_VERSION: 0.7.0-SNAPSHOT
-SPARK_VERSION_SHORT: 0.7.0
-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 9804d449fc9dd87f2fd3768e948d135ffe088daa..f06ab2d5b08ccf70cbca7c6f7a7ba34d0e0b9945 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -23,6 +23,22 @@
         <script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
         
         <link rel="stylesheet" href="css/pygments-default.css">
+
+        <!-- Google analytics script -->
+        <script type="text/javascript">
+          /*
+          var _gaq = _gaq || [];
+          _gaq.push(['_setAccount', 'UA-32518208-1']);
+          _gaq.push(['_trackPageview']);
+
+          (function() {
+            var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
+            ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
+            var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
+          })();
+          */
+        </script>
+
     </head>
     <body>
         <!--[if lt IE 7]>
@@ -48,14 +64,16 @@
                                 <li><a href="scala-programming-guide.html">Scala</a></li>
                                 <li><a href="java-programming-guide.html">Java</a></li>
                                 <li><a href="python-programming-guide.html">Python</a></li>
+                                <li><a href="streaming-programming-guide.html">Spark Streaming</a></li>
                             </ul>
                         </li>
-
+                        
                         <li class="dropdown">
-                            <a href="#" class="dropdown-toggle" data-toggle="dropdown">API<b class="caret"></b></a>
+                            <a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a>
                             <ul class="dropdown-menu">
-                                <li><a href="api/core/index.html">Scala/Java (Scaladoc)</a></li>
-                                <li><a href="api/pyspark/index.html">Python (Epydoc)</a></li>
+                                <li><a href="api/core/index.html">Spark Java/Scala (Scaladoc)</a></li>
+                                <li><a href="api/pyspark/index.html">Spark Python (Epydoc)</a></li>
+                                <li><a href="api/streaming/index.html">Spark Streaming Java/Scala (Scaladoc) </a></li>
                             </ul>
                         </li>
 
@@ -72,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 c9ce589c1b40cdf9c32650e02e954d11558571aa..d77e53963c2f7594e5857103bf062c810b861023 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -1,8 +1,9 @@
 require 'fileutils'
 include FileUtils
 
-if ENV['SKIP_SCALADOC'] != '1'
-  projects = ["core", "examples", "repl", "bagel"]
+if ENV['SKIP_API'] != '1'
+  # Build Scaladoc for Java/Scala
+  projects = ["core", "examples", "repl", "bagel", "streaming"]
 
   puts "Moving to project root and building scaladoc."
   curr_dir = pwd
@@ -11,7 +12,7 @@ if ENV['SKIP_SCALADOC'] != '1'
   puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..."
   puts `sbt/sbt doc`
 
-  puts "moving back into docs dir."
+  puts "Moving back into docs dir."
   cd("docs")
 
   # Copy over the scaladoc from each project into the docs directory.
@@ -27,9 +28,8 @@ if ENV['SKIP_SCALADOC'] != '1'
     puts "cp -r " + source + "/. " + dest
     cp_r(source + "/.", dest)
   end
-end
 
-if ENV['SKIP_EPYDOC'] != '1'
+  # Build Epydoc for Python
   puts "Moving to python directory and building epydoc."
   cd("../python")
   puts `epydoc --config epydoc.conf`
diff --git a/docs/api.md b/docs/api.md
index b9c93ac5e8cd5abe51832c1dfc0264e87d2d9184..e86d07770a80be61dd0831606421e3ee7d73d284 100644
--- a/docs/api.md
+++ b/docs/api.md
@@ -5,7 +5,8 @@ title: Spark API documentation (Scaladoc)
 
 Here you can find links to the Scaladoc generated for the Spark sbt subprojects.  If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory.
 
-- [Core](api/core/index.html)
-- [Examples](api/examples/index.html)
+- [Spark](api/core/index.html)
+- [Spark Examples](api/examples/index.html)
+- [Spark Streaming](api/streaming/index.html)
 - [Bagel](api/bagel/index.html)
 - [PySpark](api/pyspark/index.html)
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/configuration.md b/docs/configuration.md
index d8317ea97c32e96efbdc72b893a3a80de467d8a3..17fdbf04d1ca07bd133c8f7d99d210a54ba4f145 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -133,6 +133,13 @@ Apart from these, the following properties are also available, and may be useful
     it if you configure your own old generation size.
   </td>
 </tr>
+<tr>
+  <td>spark.ui.port</td>
+  <td>(random)</td>
+  <td>
+    Port for your application's dashboard, which shows memory usage of each RDD.
+  </td>
+</tr>
 <tr>
   <td>spark.shuffle.compress</td>
   <td>true</td>
@@ -183,7 +190,7 @@ Apart from these, the following properties are also available, and may be useful
 </tr>
 <tr>
   <td>spark.broadcast.factory</td>
-  <td>spark.broadcast. HttpBroadcastFactory</td>
+  <td>spark.broadcast.HttpBroadcastFactory</td>
   <td>
     Which broadcast implementation to use.
   </td>
@@ -197,28 +204,63 @@ Apart from these, the following properties are also available, and may be useful
     poor data locality, but the default generally works well.
   </td>
 </tr>
+<tr>
+  <td>spark.worker.timeout</td>
+  <td>60</td>
+  <td>
+    Number of seconds after which the standalone deploy master considers a worker lost if it
+    receives no heartbeats.
+  </td>
+</tr>
+<tr>
+  <td>spark.akka.frameSize</td>
+  <td>10</td>
+  <td>
+    Maximum message size to allow in "control plane" communication (for serialized tasks and task
+    results), in MB. Increase this if your tasks need to send back large results to the driver
+    (e.g. using <code>collect()</code> on a large dataset).
+  </td>
+</tr>
 <tr>
   <td>spark.akka.threads</td>
   <td>4</td>
   <td>
     Number of actor threads to use for communication. Can be useful to increase on large clusters
-    when the master has a lot of CPU cores.
+    when the driver has a lot of CPU cores.
+  </td>
+</tr>
+<tr>
+  <td>spark.akka.timeout</td>
+  <td>20</td>
+  <td>
+    Communication timeout between Spark nodes, in seconds.
   </td>
 </tr>
 <tr>
-  <td>spark.master.host</td>
+  <td>spark.driver.host</td>
   <td>(local hostname)</td>
   <td>
-    Hostname or IP address for the master to listen on.
+    Hostname or IP address for the driver to listen on.
   </td>
 </tr>
 <tr>
-  <td>spark.master.port</td>
+  <td>spark.driver.port</td>
   <td>(random)</td>
   <td>
-    Port for the master to listen on.
+    Port for the driver to listen on.
   </td>
 </tr>
+<tr>
+  <td>spark.cleaner.ttl</td>
+  <td>(disable)</td>
+  <td>
+    Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
+    Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is
+    useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming
+    applications). Note that any RDD that persists in memory for more than this duration will be cleared as well.
+  </td>
+</tr>
+
 </table>
 
 # Configuring Logging
diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md
index c6e01c62d808041400913a305f9346c4dd8968d3..50feeb2d6c42afa14e7092a7902d75e5362d570f 100644
--- a/docs/contributing-to-spark.md
+++ b/docs/contributing-to-spark.md
@@ -15,7 +15,7 @@ The Spark team welcomes contributions in the form of GitHub pull requests. Here
   But first, make sure that you have [configured a spark-env.sh](configuration.html) with at least
   `SCALA_HOME`, as some of the tests try to spawn subprocesses using this.
 - Add new unit tests for your code. We use [ScalaTest](http://www.scalatest.org/) for testing. Just add a new Suite in `core/src/test`, or methods to an existing Suite.
-- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issues page](https://github.com/mesos/spark/issues), or email the [mailing list](http://www.spark-project.org/mailing-lists.html).
+- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issue tracker]({{site.SPARK_ISSUE_TRACKER_URL}}), or email the [mailing list](http://www.spark-project.org/mailing-lists.html).
 
 # Licensing of Contributions
 
diff --git a/docs/css/bootstrap.css b/docs/css/bootstrap.css
index 0664207aeebb76899b248184a65d9ca64b383aa6..b51ef7b42b4383a697a14c5bfcd6713a75888c6d 100644
--- a/docs/css/bootstrap.css
+++ b/docs/css/bootstrap.css
@@ -2719,16 +2719,16 @@ table .span24 {
 .dropdown-menu .active > a:hover {
   color: #ffffff;
   text-decoration: none;
-  background-color: #0088cc;
-  background-color: #0081c2;
-  background-image: linear-gradient(to bottom, #0088cc, #0077b3);
-  background-image: -moz-linear-gradient(top, #0088cc, #0077b3);
-  background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));
-  background-image: -webkit-linear-gradient(top, #0088cc, #0077b3);
-  background-image: -o-linear-gradient(top, #0088cc, #0077b3);
+  background-color: #0098cc;
+  background-color: #0098cc;
+  background-image: linear-gradient(to bottom, #0098cc, #0087b3);
+  background-image: -moz-linear-gradient(top, #0098cc, #0087b3);
+  background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0098cc), to(#0087b3));
+  background-image: -webkit-linear-gradient(top, #0098cc, #0087b3);
+  background-image: -o-linear-gradient(top, #0098cc, #0087b3);
   background-repeat: repeat-x;
   outline: 0;
-  filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);
+  filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc', endColorstr='#ff0087b3', GradientType=0);
 }
 
 .dropdown-menu .disabled > a,
@@ -4025,18 +4025,18 @@ input[type="submit"].btn.btn-mini {
   min-height: 40px;
   padding-right: 20px;
   padding-left: 20px;
-  background-color: #fafafa;
-  background-image: -moz-linear-gradient(top, #ffffff, #f2f2f2);
-  background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2));
-  background-image: -webkit-linear-gradient(top, #ffffff, #f2f2f2);
-  background-image: -o-linear-gradient(top, #ffffff, #f2f2f2);
-  background-image: linear-gradient(to bottom, #ffffff, #f2f2f2);
+  background-color: #fadafa;
+  background-image: -moz-linear-gradient(top, #ffddff, #f2d2f2);
+  background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffddff), to(#f2d2f2));
+  background-image: -webkit-linear-gradient(top, #ffddff, #f2d2f2);
+  background-image: -o-linear-gradient(top, #ffddff, #f2d2f2);
+  background-image: linear-gradient(to bottom, #ffddff, #f2d2f2);
   background-repeat: repeat-x;
   border: 1px solid #d4d4d4;
   -webkit-border-radius: 4px;
      -moz-border-radius: 4px;
           border-radius: 4px;
-  filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff2f2f2', GradientType=0);
+  filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff', endColorstr='#fff2d2f2', GradientType=0);
   -webkit-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065);
      -moz-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065);
           box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065);
diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css
index 3119038a6b7ab35240946fbe710ec68c996fbd29..eb48138e08d686fb220b74b238f2ae9b53718103 100644
--- a/docs/css/bootstrap.min.css
+++ b/docs/css/bootstrap.min.css
@@ -6,4 +6,4 @@
  * http://www.apache.org/licenses/LICENSE-2.0
  *
  * Designed and built with all the love in the world @twitter by @mdo and @fat.
- */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#08c;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#08c;background-color:#0081c2;background-image:linear-gradient(to bottom,#08c,#0077b3);background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fff2f2f2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
+ */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0098cc;background-color:#0098cc;background-image:-moz-linear-gradient(top,#0098cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0098cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0098cc,#0087b3);background-image:-o-linear-gradient(top,#0098cc,#0087b3);background-image:linear-gradient(to bottom,#0098cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0098cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0098cc,#0087b3);background-image:-moz-linear-gradient(top,#0098cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0098cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0098cc,#0087b3);background-image:-o-linear-gradient(top,#0098cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0098cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#e2f2e2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e2f2e2));background-image:-webkit-linear-gradient(top,#fff,#e2f2e2);background-image:-o-linear-gradient(top,#fff,#e2f2e2);background-image:linear-gradient(to bottom,#fff,#e2f2e2);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffddff',endColorstr='#ffe2f2e2',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md
index 6e1f7fd3b1d6208b4f297ccbfe84f45c21156a7b..dc57035ebaf6cba0fcab6fcdd3934ca44fc3c6c5 100644
--- a/docs/ec2-scripts.md
+++ b/docs/ec2-scripts.md
@@ -45,9 +45,9 @@ identify machines belonging to each cluster in the Amazon EC2 Console.
     key pair, `<num-slaves>` is the number of slave nodes to launch (try
     1 at first), and `<cluster-name>` is the name to give to your
     cluster.
--   After everything launches, check that Mesos is up and sees all the
-    slaves by going to the Mesos Web UI link printed at the end of the
-    script (`http://<master-hostname>:8080`).
+-   After everything launches, check that the cluster scheduler is up and sees
+    all the slaves by going to its web UI, which will be printed at the end of
+    the script (typically `http://<master-hostname>:8080`).
 
 You can also run `./spark-ec2 --help` to see more usage options. The
 following options are worth pointing out:
@@ -68,6 +68,9 @@ available.
 -    `--ebs-vol-size=GB` will attach an EBS volume with a given amount
      of space to each node so that you can have a persistent HDFS cluster
      on your nodes across cluster restarts (see below).
+-    `--spot-price=PRICE` will launch the worker nodes as
+     [Spot Instances](http://aws.amazon.com/ec2/spot-instances/),
+     bidding for the given maximum price (in dollars).
 -    If one of your launches fails due to e.g. not having the right
 permissions on your private key file, you can run `launch` with the
 `--resume` option to restart the setup process on an existing cluster.
@@ -80,7 +83,7 @@ permissions on your private key file, you can run `launch` with the
     above. (This is just for convenience; you could also use
     the EC2 console.)
 -   To deploy code or data within your cluster, you can log in and use the
-    provided script `~/mesos-ec2/copy-dir`, which,
+    provided script `~/spark-ec2/copy-dir`, which,
     given a directory path, RSYNCs it to the same location on all the slaves.
 -   If your job needs to access large datasets, the fastest way to do
     that is to load them from Amazon S3 or an Amazon EBS device into an
@@ -96,7 +99,9 @@ permissions on your private key file, you can run `launch` with the
     `spark-ec2` to attach a persistent EBS volume to each node for
     storing the persistent HDFS.
 -   Finally, if you get errors while running your jobs, look at the slave's logs
-    for that job using the Mesos web UI (`http://<master-hostname>:8080`).
+    for that job inside of the Mesos work directory (/mnt/mesos-work). You can
+    also view the status of the cluster using the Mesos web UI 
+    (`http://<master-hostname>:8080`).
 
 # Configuration
 
@@ -104,7 +109,7 @@ You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark config
 as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`).
 This file needs to be copied to **every machine** to reflect the change. The easiest way to do this
 is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master, 
-then run `~/mesos-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers.
+then run `~/spark-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers.
 
 The [configuration guide](configuration.html) describes the available configuration options.
 
@@ -150,10 +155,10 @@ If you have a patch or suggestion for one of these limitations, feel free to
 
 # Using a Newer Spark Version
 
-The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/mesos-ec2/copy-dir /root/spark`.
+The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/spark-ec2/copy-dir /root/spark`.
 
 # Accessing Data in S3
 
-Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://<id>:<secret>@<bucket>/path`, where `<id>` is your Amazon access key ID and `<secret>` is your Amazon secret access key. Note that you should escape any `/` characters in the secret key as `%2F`. Full instructions can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3).
+Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://<bucket>/path`. You will also need to set your Amazon security credentials, either by setting the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` before your program or through `SparkContext.hadoopConfiguration`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3).
 
 In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory.
diff --git a/docs/index.md b/docs/index.md
index 848b58533359e7cb1e2a83a0f749f9fe2e54d9d5..0c4add45dcd2adb6a5b4783a670afb985a6b8ce0 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -3,15 +3,9 @@ layout: global
 title: Spark Overview
 ---
 
-{% comment %}
-TODO(andyk): Rewrite to make the Java API a first class part of the story.
-{% endcomment %}
-
 Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter.
 It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators.
-Spark can run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager,
-[Hadoop YARN](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html),
-Amazon EC2, or without an independent resource manager ("standalone mode").
+Spark can run on the Apache Mesos cluster manager, Hadoop YARN, Amazon EC2, or without an independent resource manager ("standalone mode").
 
 # Downloading
 
@@ -24,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.
@@ -60,11 +56,13 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
 * [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API
 * [Java Programming Guide](java-programming-guide.html): using Spark from Java
 * [Python Programming Guide](python-programming-guide.html): using Spark from Python
+* [Spark Streaming Guide](streaming-programming-guide.html): using the alpha release of Spark Streaming
 
 **API Docs:**
 
-* [Java/Scala (Scaladoc)](api/core/index.html)
-* [Python (Epydoc)](api/pyspark/index.html)
+* [Spark Java/Scala (Scaladoc)](api/core/index.html)
+* [Spark Python (Epydoc)](api/pyspark/index.html)
+* [Spark Streaming Java/Scala (Scaladoc)](api/streaming/index.html)
 
 **Deployment guides:**
 
@@ -76,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
@@ -90,7 +89,8 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
   [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are
   available online for free.
 * [Code Examples](http://spark-project.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark
-* [Paper Describing the Spark System](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf)
+* [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf)
+* [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
 
 # Community
 
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
index 188ca4995ecadb509a76293be223dcca9666bcff..ae8257b53938e0672efc1cb35d4f77893a7a1ec2 100644
--- a/docs/java-programming-guide.md
+++ b/docs/java-programming-guide.md
@@ -75,7 +75,8 @@ class has a single abstract method, `call()`, that must be implemented.
 ## Storage Levels
 
 RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are
-declared in the [spark.api.java.StorageLevels](api/core/index.html#spark.api.java.StorageLevels) class.
+declared in the [spark.api.java.StorageLevels](api/core/index.html#spark.api.java.StorageLevels) class. To
+define your own storage level, you can use StorageLevels.create(...). 
 
 
 # Other Features
@@ -188,7 +189,7 @@ We hope to generate documentation with Java-style syntax in the future.
 # Where to Go from Here
 
 Spark includes several sample programs using the Java API in
-`examples/src/main/java`.  You can run them by passing the class name to the
+[`examples/src/main/java`](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/examples).  You can run them by passing the class name to the
 `run` script included in Spark -- for example, `./run
 spark.examples.JavaWordCount`.  Each example program prints usage help when run
 without any arguments.
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 78ef310a004ad63328586be702a8fc710d4c98ab..3a7a8db4a6ee43fdfa7af612c39a09b953b6560a 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -16,7 +16,6 @@ There are a few key differences between the Python and Scala APIs:
 
 * Python is dynamically typed, so RDDs can hold objects of different types.
 * PySpark does not currently support the following Spark features:
-    - Accumulators
     - Special functions on RDDs of doubles, such as `mean` and `stdev`
     - `lookup`
     - `persist` at storage levels other than `MEMORY_ONLY`
@@ -68,20 +67,27 @@ The script automatically adds the `pyspark` package to the `PYTHONPATH`.
 
 # Interactive Use
 
-The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs.
-When run without any input files, `pyspark` launches a shell that can be used explore data interactively, which is a simple way to learn the API:
+The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
+
+{% highlight bash %}
+$ sbt/sbt package
+$ ./pyspark
+{% endhighlight %}
+
+The Python shell can be used explore data interactively and is a simple way to learn the API:
 
 {% highlight python %}
 >>> words = sc.textFile("/usr/share/dict/words")
 >>> words.filter(lambda w: w.startswith("spar")).take(5)
 [u'spar', u'sparable', u'sparada', u'sparadrap', u'sparagrass']
+>>> help(pyspark) # Show all pyspark functions
 {% endhighlight %}
 
 By default, the `pyspark` shell creates SparkContext that runs jobs locally.
 To connect to a non-local cluster, set the `MASTER` environment variable.
 For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
 
-{% highlight shell %}
+{% highlight bash %}
 $ MASTER=spark://IP:PORT ./pyspark
 {% endhighlight %}
 
@@ -103,9 +109,9 @@ Code dependencies can be added to an existing SparkContext using its `addPyFile(
 
 # Where to Go from Here
 
-PySpark includes several sample programs using the Python API in `python/examples`.
+PySpark includes several sample programs in the [`python/examples` folder](https://github.com/mesos/spark/tree/master/python/examples).
 You can run them by passing the files to the `pyspark` script -- for example `./pyspark python/examples/wordcount.py`.
-Each example program prints usage help when run without any arguments.
+Each program prints usage help when run without arguments.
 
 We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc.
 Many of the RDD method descriptions contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index a4c4c9a8fb0e30831819c15c7e50fd68229a0f98..2d961b29cb197feb6a23a71ca9174fa9a814aa7d 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,14 +111,16 @@ 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 = "/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))
+  }
 }
 {% endhighlight %}
 
@@ -136,7 +138,7 @@ scalaVersion := "{{site.SCALA_VERSION}}"
 libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}"
 
 resolvers ++= Seq(
-  "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
+  "Akka Repository" at "http://repo.akka.io/releases/",
   "Spray Repository" at "http://repo.spray.cc/")
 {% endhighlight %}
 
@@ -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 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.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.
 
@@ -207,8 +209,8 @@ To build the job, we also write a Maven `pom.xml` file that lists Spark as a dep
       <url>http://repo.spray.cc</url>
     </repository>
     <repository>
-      <id>Typesafe repository</id>
-      <url>http://repo.typesafe.com/typesafe/releases</url>
+      <id>Akka repository</id>
+      <url>http://repo.akka.io/releases</url>
     </repository>
   </repositories>
   <dependencies>
@@ -265,7 +267,7 @@ 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.
 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:
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 6fb81b60043f9b7ecdb8c59c25bdb673e7927c7e..c2957e6cb42eb7863280f9b324e244578ba145f8 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -4,7 +4,7 @@ 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)
+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.  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:
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 7350eca83796f2a82ab7bd020cd03a783d424279..2315aadbdf17ce5dcc70d17518b275c496cee001 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -38,10 +38,10 @@ The first thing a Spark program must do is to create a `SparkContext` object, wh
 This is done through the following constructor:
 
 {% highlight scala %}
-new SparkContext(master, jobName, [sparkHome], [jars])
+new SparkContext(master, appName, [sparkHome], [jars])
 {% endhighlight %}
 
-The `master` parameter is a string specifying a [Mesos](running-on-mesos.html) cluster to connect to, or a special "local" string to run in local mode, as described below. `jobName` is a name for your job, which will be shown in the Mesos web UI when running on a cluster. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
+The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
 
 In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable. For example, to run on four cores, use
 
@@ -203,7 +203,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in
 <tr><th>Action</th><th>Meaning</th></tr>
 <tr>
   <td> <b>reduce</b>(<i>func</i>) </td>
-  <td> Aggregate the elements of the dataset using a function <i>func</i> (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. </td>
+  <td> Aggregate the elements of the dataset using a function <i>func</i> (which takes two arguments and returns one). The function should be commutative and associative so that it can be computed correctly in parallel. </td>
 </tr>
 <tr>
   <td> <b>collect</b>() </td>
@@ -301,7 +301,8 @@ We recommend going through the following process to select one:
 * Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web
   application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones
   let you continue running tasks on the RDD without waiting to recompute a lost partition.
-  
+ 
+If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#spark.storage.StorageLevel$) singleton object.  
 
 # Shared Variables
 
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index e0ba7c35cb1fa4f21d77030e5c4fcb0734a13943..3986c0c79dceaecb0da46adbf07241b64229f650 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -51,11 +51,11 @@ Finally, the following configuration options can be passed to the master and wor
   </tr>
   <tr>
     <td><code>-c CORES</code>, <code>--cores CORES</code></td>
-    <td>Number of CPU cores to use (default: all available); only on worker</td>
+    <td>Total CPU cores to allow Spark jobs to use on the machine (default: all available); only on worker</td>
   </tr>
   <tr>
     <td><code>-m MEM</code>, <code>--memory MEM</code></td>
-    <td>Amount of memory to use, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
+    <td>Total amount of memory to allow Spark jobs to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
   </tr>
   <tr>
     <td><code>-d DIR</code>, <code>--work-dir DIR</code></td>
@@ -66,9 +66,20 @@ Finally, the following configuration options can be passed to the master and wor
 
 # Cluster Launch Scripts
 
-To launch a Spark standalone cluster with the deploy scripts, you need to set up two files, `conf/spark-env.sh` and `conf/slaves`. The `conf/spark-env.sh` file lets you specify global settings for the master and slave instances, such as memory, or port numbers to bind to, while `conf/slaves` is a list of slave nodes. The system requires that all the slave machines have the same configuration files, so *copy these files to each machine*.
+To launch a Spark standalone cluster with the deploy scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file.
 
-In `conf/spark-env.sh`, you can set the following parameters, in addition to the [standard Spark configuration settings](configuration.html):
+Once you've set up this fine, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
+
+- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
+- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
+- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
+- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
+- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
+- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
+
+Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.
+
+You can optionally configure the cluster further by setting environment variables in `conf/spark-env.sh`. Create this file by starting with the `conf/spark-env.sh.template`, and _copy it to all your worker machines_ for the settings to take effect. The following settings are available:
 
 <table class="table">
   <tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
@@ -88,36 +99,32 @@ In `conf/spark-env.sh`, you can set the following parameters, in addition to the
     <td><code>SPARK_WORKER_PORT</code></td>
     <td>Start the Spark worker on a specific port (default: random)</td>
   </tr>
+  <tr>
+    <td><code>SPARK_WORKER_DIR</code></td>
+    <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work)</td>
+  </tr>
   <tr>
     <td><code>SPARK_WORKER_CORES</code></td>
-    <td>Number of cores to use (default: all available cores)</td>
+    <td>Total number of cores to allow Spark jobs to use on the machine (default: all available cores)</td>
   </tr>
   <tr>
     <td><code>SPARK_WORKER_MEMORY</code></td>
-    <td>How much memory to use, e.g. 1000M, 2G (default: total memory minus 1 GB)</td>
+    <td>Total amount of memory to allow Spark jobs to use on the machine, e.g. 1000M, 2G (default: total memory minus 1 GB); note that each job's <i>individual</i> memory is configured using <code>SPARK_MEM</code></td>
   </tr>
   <tr>
     <td><code>SPARK_WORKER_WEBUI_PORT</code></td>
     <td>Port for the worker web UI (default: 8081)</td>
   </tr>
   <tr>
-    <td><code>SPARK_WORKER_DIR</code></td>
-    <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work)</td>
+    <td><code>SPARK_DAEMON_MEMORY</code></td>
+    <td>Memory to allocate to the Spark master and worker daemons themselves (default: 512m)</td>
+  </tr>
+  <tr>
+    <td><code>SPARK_DAEMON_JAVA_OPTS</code></td>
+    <td>JVM options for the Spark master and worker daemons themselves (default: none)</td>
   </tr>
 </table>
 
-In `conf/slaves`, include a list of all machines where you would like to start a Spark worker, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing purposes, you can have a single `localhost` entry in the slaves file.
-
-Once you've set up these configuration files, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
-
-- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
-- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
-- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
-- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
-- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
-- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
-
-Note that the scripts must be executed on the machine you want to run the Spark master on, not your local machine.
 
 
 # Connecting a Job to the Cluster
diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md
new file mode 100644
index 0000000000000000000000000000000000000000..5476c00d020cb2cfece90853d7e010d08055c00d
--- /dev/null
+++ b/docs/streaming-custom-receivers.md
@@ -0,0 +1,101 @@
+---
+layout: global
+title: Tutorial - Spark Streaming, Plugging in a custom receiver.
+---
+
+A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need.
+
+This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application.
+
+
+## A quick and naive walk-through
+
+### Write a simple receiver
+
+This starts with implementing [Actor](#References)
+
+Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api.
+
+{% highlight scala %}
+
+       class SocketTextStreamReceiver (host:String,
+         port:Int,
+         bytesToString: ByteString => String) extends Actor with Receiver {
+
+          override def preStart = IOManager(context.system).connect(host, port)
+
+          def receive = {
+           case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes))
+         }
+
+       }
+
+
+{% endhighlight %}
+
+All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details.
+
+### A sample spark application
+
+* First create a Spark streaming context with master url and batchduration.
+
+{% highlight scala %}
+
+    val ssc = new StreamingContext(master, "WordCountCustomStreamSource",
+      Seconds(batchDuration))
+
+{% endhighlight %}
+
+* Plug-in the actor configuration into the spark streaming context and create a DStream.
+
+{% highlight scala %}
+
+    val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
+      "localhost",8445, z => z.utf8String)),"SocketReceiver")
+
+{% endhighlight %}
+
+* Process it.
+
+{% highlight scala %}
+
+    val words = lines.flatMap(_.split(" "))
+    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+
+    wordCounts.print()
+    ssc.start()
+
+
+{% endhighlight %}
+
+* After processing it, stream can be tested using the netcat utility.
+
+     $ nc -l localhost 8445
+     hello world
+     hello hello
+
+
+## Multiple homogeneous/heterogeneous receivers.
+
+A DStream union operation is provided for taking union on multiple input streams.
+
+{% highlight scala %}
+
+    val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
+      "localhost",8445, z => z.utf8String)),"SocketReceiver")
+
+    // Another socket stream receiver
+    val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
+      "localhost",8446, z => z.utf8String)),"SocketReceiver")
+
+    val union = lines.union(lines2)
+
+{% endhighlight %}
+
+Above stream can be easily process as described earlier.
+
+_A more comprehensive example is provided in the spark streaming examples_
+
+## References
+
+1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html)
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
new file mode 100644
index 0000000000000000000000000000000000000000..f5788dc46776d6709f1991195aefe2e742a35841
--- /dev/null
+++ b/docs/streaming-programming-guide.md
@@ -0,0 +1,517 @@
+---
+layout: global
+title: Spark Streaming Programming Guide
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+# Overview
+A Spark Streaming application is very similar to a Spark application; it consists of a *driver program* that runs the user's `main` function and continuous executes various *parallel operations* on input streams of data. The main abstraction Spark Streaming provides is a *discretized stream* (DStream), which is a continuous sequence of RDDs (distributed collections of elements) representing a continuous stream of data. DStreams can be created from live incoming data (such as data from a socket, Kafka, etc.) or can be generated by transformong existing DStreams using parallel operators like `map`, `reduce`, and `window`. The basic processing model is as follows: 
+(i) While a Spark Streaming driver program is running, the system receives data from various sources and and divides it into batches. Each batch of data is treated as an RDD, that is, an immutable parallel collection of data. These input RDDs are saved in memory and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively called an InputDStream.
+(ii) Data received by InputDStreams are processed using DStream operations. Since all data is represented as RDDs and all DStream operations as RDD operations, data is automatically recovered in the event of node failures.  
+
+This guide shows some how to start programming with DStreams. 
+
+# Initializing Spark Streaming
+The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using
+
+{% highlight scala %}
+new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
+{% endhighlight %}
+
+The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
+
+This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`.
+
+
+# Attaching Input Sources - InputDStreams
+The StreamingContext is used to creating InputDStreams from input sources:
+
+{% highlight scala %}
+// Assuming ssc is the StreamingContext
+ssc.textFileStream(directory)      // Creates a stream by monitoring and processing new files in a HDFS directory
+ssc.socketStream(hostname, port)   // Creates a stream that uses a TCP socket to read data from hostname:port
+{% endhighlight %}
+
+We also provide a input streams for Kafka, Flume, Akka actor, etc. For a complete list of input streams, take a look at the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext).
+
+
+
+# DStream Operations
+Data received from the input streams can be processed using _DStream operations_. There are two kinds of operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams with transformed data. After applying a sequence of transformations to the input streams, you'll need to call the output operations, which writies data out to an external source. 
+
+## Transformations
+
+DStreams support many of the transformations available on normal Spark RDD's:
+
+<table class="table">
+<tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
+<tr>
+  <td> <b>map</b>(<i>func</i>) </td>
+  <td> Returns a new DStream formed by passing each element of the source DStream through a function <i>func</i>. </td>
+</tr>
+<tr>
+  <td> <b>filter</b>(<i>func</i>) </td>
+  <td> Returns a new DStream formed by selecting those elements of the source DStream on which <i>func</i> returns true. </td>
+</tr>
+<tr>
+  <td> <b>flatMap</b>(<i>func</i>) </td>
+  <td> Similar to map, but each input item can be mapped to 0 or more output items (so <i>func</i> should return a <code>Seq</code> rather than a single item). </td>
+</tr>
+<tr>
+  <td> <b>mapPartitions</b>(<i>func</i>) </td>
+  <td> Similar to map, but runs separately on each partition (block) of the DStream, so <i>func</i> must be of type
+    Iterator[T] => Iterator[U] when running on an DStream of type T. </td>
+</tr>
+<tr>
+  <td> <b>union</b>(<i>otherStream</i>) </td>
+  <td> Return a new DStream that contains the union of the elements in the source DStream and the argument DStream. </td>
+</tr>
+<tr>
+  <td> <b>count</b>() </td>
+  <td> Returns a new DStream of single-element RDDs by counting the number of elements in each RDD of the source DStream.  </td>
+</tr>
+<tr>
+  <td> <b>reduce</b>(<i>func</i>) </td>
+  <td> Returns a new DStream of single-element RDDs by aggregating the elements in each RDD of the source DStream using a function <i>func</i> (which takes two arguments and returns one). The function should be associative so that it can be computed in parallel. </td>
+</tr>
+<tr>
+  <td> <b>countByValue</b>() </td>
+  <td> When called on a DStream of elements of type K, returns a new DStream of (K, Long) pairs where the value of each key is its frequency in each RDD of the source DStream.  </td>
+</tr>
+<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 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>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td>
+  <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in <code>groupByKey</code>, the number of reduce tasks is configurable through an optional second argument. </td>
+</tr>
+<tr>
+  <td> <b>join</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td>
+  <td> When called on two DStreams of (K, V) and (K, W) pairs, returns a new DStream of (K, (V, W)) pairs with all pairs of elements for each key. </td>
+</tr>
+<tr>
+  <td> <b>cogroup</b>(<i>otherStream</i>, [<i>numTasks</i>]) </td>
+  <td> When called on DStream of (K, V) and (K, W) pairs, returns a new DStream of (K, Seq[V], Seq[W]) tuples.</td>
+</tr>
+<tr>
+  <td> <b>transform</b>(<i>func</i>) </td>
+  <td> Returns a new DStream by applying func (a RDD-to-RDD function) to every RDD of the stream. This can be used to do arbitrary RDD operations on the DStream. </td>
+</tr>
+<tr>
+  <td> <b>updateStateByKey</b>(<i>func</i>) </td>
+  <td> Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values of each key. This can be used to track session state by using the session-id as the key and updating the session state as new data is received.</td>
+</tr>
+
+</table>
+
+Spark Streaming features windowed computations, which allow you to apply transformations over a sliding window of data. All window functions take a <i>windowDuration</i>, which represents the width of the window and a <i>slideTime</i>, which represents the frequency during which the window is calculated.
+
+<table class="table">
+<tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
+<tr>
+  <td> <b>window</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <td> Return a new DStream which is computed based on windowed batches of the source DStream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
+  </td>
+</tr>
+<tr>
+  <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
+  </td>
+</tr>
+<tr>
+  <td> <b>reduceByWindow</b>(<i>func</i>, <i>windowDuration</i>, <i>slideDuration</i>) </td>
+  <td> Return a new single-element stream, created by aggregating elements in the stream over a sliding interval using <i>func</i>. The function should be associative so that it can be computed correctly in parallel. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
+  </td>
+</tr>
+<tr>
+  <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 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>
+  <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, V) pairs where the values for each key are aggregated using the given reduce function <i>func</i> over batches in a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
+ <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
+</td> 
+</tr>
+<tr>
+  <td> <b>reduceByKeyAndWindow</b>(<i>func</i>, <i>invFunc</i>, <i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>]) </td>
+  <td> A more efficient version of the above <code>reduceByKeyAndWindow()</code> where the reduce value of each window is calculated
+  incrementally using the reduce values of the previous window. This is done by reducing the new data that enter the sliding window, and "inverse reducing" the old data that leave the window. An example would be that of "adding" and "subtracting" counts of keys as the window slides. However, it is applicable to only "invertible reduce functions", that is, those reduce functions which have a corresponding "inverse reduce" function (taken as parameter <i>invFunc</i>. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
+ <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
+</td>
+</tr>
+<tr>
+  <td> <b>countByValueAndWindow</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, Long) pairs where the value of each key is its frequency within a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument.
+ <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
+</td>
+</tr>
+
+</table>
+
+A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#spark.streaming.PairDStreamFunctions).
+
+## Output Operations
+When an output operator is called, it triggers the computation of a stream. Currently the following output operators are defined:
+
+<table class="table">
+<tr><th style="width:30%">Operator</th><th>Meaning</th></tr>
+<tr>
+  <td> <b>foreach</b>(<i>func</i>) </td>
+  <td> The fundamental output operator. Applies a function, <i>func</i>, to each RDD generated from the stream. This function should have side effects, such as printing output, saving the RDD to external files, or writing it over the network to an external system. </td>
+</tr>
+
+<tr>
+  <td> <b>print</b>() </td>
+  <td> Prints first ten elements of every batch of data in a DStream on the driver. </td>
+</tr>
+
+<tr>
+  <td> <b>saveAsObjectFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+  <td> Save this DStream's contents as a <code>SequenceFile</code> of serialized objects. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>.
+  </td>
+</tr>
+
+<tr>
+  <td> <b>saveAsTextFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+  <td> Save this DStream's contents as a text files. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
+</tr>
+
+<tr>
+  <td> <b>saveAsHadoopFiles</b>(<i>prefix</i>, [<i>suffix</i>]) </td>
+  <td> Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on <i>prefix</i> and <i>suffix</i>: <i>"prefix-TIME_IN_MS[.suffix]"</i>. </td>
+</tr>
+
+</table>
+
+# Starting the Streaming computation
+All the above DStream operations are completely lazy, that is, the operations will start executing only after the context is started by using
+{% highlight scala %}
+ssc.start()
+{% endhighlight %}
+
+Conversely, the computation can be stopped by using
+{% highlight scala %}
+ssc.stop()
+{% endhighlight %}
+
+# Example
+A simple example to start off is the [NetworkWordCount](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` .
+
+{% highlight scala %}
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+...
+
+// Create the context and set up a network input stream to receive from a host:port
+val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1))
+val lines = ssc.socketTextStream(args(1), args(2).toInt)
+
+// Split the lines into words, count them, and print some of the counts on the master
+val words = lines.flatMap(_.split(" "))
+val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+wordCounts.print()
+
+// Start the computation
+ssc.start()
+{% endhighlight %}
+
+The `socketTextStream` returns a DStream of lines received from a TCP socket-based source. The `lines` DStream is _transformed_ into a DStream using the `flatMap` operation, where each line is split into words. This `words` DStream is then mapped to a DStream of `(word, 1)` pairs, which is finally reduced to get the word counts. `wordCounts.print()` will print 10 of the counts generated every second.
+
+To run this example on your local machine, you need to first run a Netcat server by using
+
+{% highlight bash %}
+$ nc -lk 9999
+{% endhighlight %}
+
+Then, in a different terminal, you can start NetworkWordCount by using
+
+{% highlight bash %}
+$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+{% endhighlight %}
+
+This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
+
+<table>
+<td>
+{% highlight bash %}
+# TERMINAL 1
+# RUNNING NETCAT
+
+$ nc -lk 9999
+hello world
+
+
+
+
+
+...
+{% endhighlight %}
+</td>
+<td>
+{% highlight bash %}
+# TERMINAL 2: RUNNING NetworkWordCount
+...
+2012-12-31 18:47:10,446 INFO SparkContext: Job finished: run at ThreadPoolExecutor.java:886, took 0.038817 s
+-------------------------------------------
+Time: 1357008430000 ms
+-------------------------------------------
+(hello,1)
+(world,1)
+
+2012-12-31 18:47:10,447 INFO JobManager: Total delay: 0.44700 s for job 8 (execution: 0.44000 s)
+...
+{% endhighlight %}
+</td>
+</table>
+
+You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
+
+# DStream Persistence
+Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.
+
+For input streams that receive data from the network (that is, subclasses of NetworkInputDStream like FlumeInputDStream and KafkaInputDStream), the default persistence level is set to replicate the data to two nodes for fault-tolerance.
+
+Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More information on different persistence levels can be found in [Spark Programming Guide](scala-programming-guide.html#rdd-persistence).
+
+# RDD Checkpointing within DStreams
+A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. 
+
+Because stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. To clear this metadata, streaming supports periodic _checkpointing_ by saving intermediate data to HDFS. Note that checkpointing also incurs the cost of saving to HDFS which may cause the corresponding batch to take longer to process. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too slowly causes the lineage and task sizes to grow which may have detrimental effects. Typically, a checkpoint interval of 5 - 10 times of sliding interval of a DStream is good setting to try.
+
+To enable checkpointing, the developer has to provide the HDFS path to which RDD will be saved. This is done by using
+
+{% highlight scala %}
+ssc.checkpoint(hdfsPath) // assuming ssc is the StreamingContext
+{% endhighlight %}
+
+The interval of checkpointing of a DStream can be set by using
+
+{% highlight scala %}
+dstream.checkpoint(checkpointInterval) // checkpointInterval must be a multiple of slide duration of dstream
+{% endhighlight %}
+
+For DStreams that must be checkpointed (that is, DStreams created by `updateStateByKey` and `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds.
+
+
+# Performance Tuning
+Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things:
+<ol>
+<li>Reducing the processing time of each batch of data by efficiently using cluster resources.</li>
+<li>Setting the right batch size such that the data processing can keep up with the data ingestion.</li>
+</ol>
+
+## Reducing the Processing Time of each Batch
+There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
+
+### Level of Parallelism
+Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`spark.PairDStreamFunctions`](api/streaming/index.html#spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default.
+
+### Data Serialization
+The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it.
+
+* **Serialization of RDD data in Spark**: Please refer to the detailed discussion on data serialization in the [Tuning Guide](tuning.html). However, note that unlike Spark, by default RDDs are persisted as serialized byte arrays to minimize pauses related to GC.
+
+* **Serialization of input data**: To ingest external data into Spark, data received as bytes (say, from the network) needs to deserialized from bytes and re-serialized into Spark's serialization format. Hence, the deserialization overhead of input data may be a bottleneck.
+
+### Task Launching Overheads
+If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes:
+
+* **Task Serialization**: Using Kryo serialization for serializing tasks can reduced the task sizes, and therefore reduce the time taken to send them to the slaves.
+
+* **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to better task launch times than the fine-grained Mesos mode. Please refer to the [Running on Mesos guide](running-on-mesos.html) for more details.
+These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable.
+
+## Setting the Right Batch Size
+For a Spark Streaming application running on a cluster to be stable, the processing of the data streams must keep up with the rate of ingestion of the data streams. Depending on the type of computation, the batch size used may have significant impact on the rate of ingestion that can be sustained by the Spark Streaming application on a fixed cluster resources. For example, let us consider the earlier WordCountNetwork example. For a particular data rate, the system may be able to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not every 500 milliseconds.
+
+A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size).
+
+## 24/7 Operation
+By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
+
+This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so.
+
+## Memory Tuning
+Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, we highlight a few customizations that are strongly recommended to minimize GC related pauses in Spark Streaming applications and achieving more consistent batch processing times.
+
+* **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, [StorageLevel.MEMORY_ONLY_SER](api/core/index.html#spark.storage.StorageLevel$) for DStream compared to [StorageLevel.MEMORY_ONLY](api/core/index.html#spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs a higher serialization overheads, it significantly reduces GC pauses.
+
+* **Concurrent garbage collector**: Using the concurrent mark-and-sweep GC further minimizes the variability of GC pauses. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times.
+
+# Fault-tolerance Properties
+In this section, we are going to discuss the behavior of Spark Streaming application in the event of a node failure. To understand this, let us remember the basic fault-tolerance properties of Spark's RDDs.
+
+ 1. An RDD is an immutable, and deterministically re-computable, distributed dataset. Each RDD remembers the lineage of deterministic operations that were used on a fault-tolerant input dataset to create it.
+ 1. If any partition of an RDD is lost due to a worker node failure, then that partition can be re-computed from the original fault-tolerant dataset using the lineage of operations.
+
+Since all data transformations in Spark Streaming are based on RDD operations, as long as the input dataset is present, all intermediate data can recomputed. Keeping these properties in mind, we are going to discuss the failure semantics in more detail.
+
+## Failure of a Worker Node
+
+There are two failure behaviors based on which input sources are used.
+
+1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
+1. _Using any input source that receives data through a network_ - For network-based data sources like Kafka and Flume, the received input data is replicated in memory between nodes of the cluster (default replication factor is 2). So if a worker node fails, then the system can recompute the lost from the the left over copy of the input data. However, if the worker node where a network receiver was running fails, then a tiny bit of data may be lost, that is, the data received by the system but not yet replicated to other node(s). The receiver will be started on a different node and it will continue to receive data.
+
+Since all data is modeled as RDDs with their lineage of deterministic operations, any recomputation always leads to the same result. As a result, all DStream transformations are guaranteed to have _exactly-once_ semantics. That is, the final transformed result will be same even if there were was a worker node failure. However, output operations (like `foreach`) have _at-least once_ semantics, that is, the transformed data may get written to an external entity more than once in the event of a worker failure. While this is acceptable for saving to HDFS using the `saveAs*Files` operations (as the file will simply get over-written by the same data), additional transactions-like mechanisms may be necessary to achieve exactly-once semantics for output operations.
+
+## Failure of the Driver Node
+A system that is required to operate 24/7 needs to be able tolerate the failure of the driver node as well. Spark Streaming does this by saving the state of the DStream computation periodically to a HDFS file, that can be used to restart the streaming computation in the event of a failure of the driver node. This checkpointing is enabled by setting a HDFS directory for checkpointing using `ssc.checkpoint(<checkpoint directory>)` as described [earlier](#rdd-checkpointing-within-dstreams). To elaborate, the following state is periodically saved to a file.
+
+1. The DStream operator graph (input streams, output streams, etc.)
+1. The configuration of each DStream (checkpoint interval, etc.)
+1. The RDD checkpoint files of each DStream
+
+All this is periodically saved in the file `<checkpoint directory>/graph`. To recover, a new Streaming Context can be created with this directory by using
+
+{% highlight scala %}
+val ssc = new StreamingContext(checkpointDirectory)
+{% endhighlight %}
+
+On calling `ssc.start()` on this new context, the following steps are taken by the system
+
+1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it last checkpointed. This is also done for those time steps that were previously scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc.
+1. Restart the network receivers, if any, and continue receiving new data.
+
+In the current _alpha_ release, there are two different failure behaviors based on which input sources are used.
+
+1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
+1. _Using any input source that receives data through a network_ - The received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely.
+
+In future releases, we will support full recoverability for all input sources. Note that for non-stateful transformations like `map`, `count`, and `reduceByKey`, with _all_ input streams, the system, upon restarting, will continue to receive and process new data.
+
+To better understand the behavior of the system under driver failure with a HDFS source, lets consider what will happen with a file input stream Specifically, in the case of the file input stream, it will correctly identify new files that were created while the driver was down and process them in the same way as it would have if the driver had not failed. To explain further in the case of file input stream, we shall use an example. Lets say, files are being generated every second, and a Spark Streaming program reads every new file and output the number of lines in the file. This is what the sequence of outputs would be with and without a driver failure.
+
+<table class="table">
+    <!-- Results table headers -->
+    <tr>
+      <th> Time </th>
+      <th> Number of lines in input file </th>
+      <th> Output without driver failure </th>
+      <th> Output with driver failure </th>
+    </tr>
+    <tr>
+      <td>1</td>
+      <td>10</td>
+      <td>10</td>
+      <td>10</td>
+    </tr>
+    <tr>
+      <td>2</td>
+      <td>20</td>
+      <td>20</td>
+      <td>20</td>
+    </tr>
+    <tr>
+      <td>3</td>
+      <td>30</td>
+      <td>30</td>
+      <td>30</td>
+    </tr>
+    <tr>
+      <td>4</td>
+      <td>40</td>
+      <td>40</td>
+      <td>[DRIVER FAILS]<br />no output</td>
+    </tr>
+    <tr>
+      <td>5</td>
+      <td>50</td>
+      <td>50</td>
+      <td>no output</td>
+    </tr>
+    <tr>
+      <td>6</td>
+      <td>60</td>
+      <td>60</td>
+      <td>no output</td>
+    </tr>
+    <tr>
+      <td>7</td>
+      <td>70</td>
+      <td>70</td>
+      <td>[DRIVER RECOVERS]<br />40, 50, 60, 70</td>
+    </tr>
+    <tr>
+      <td>8</td>
+      <td>80</td>
+      <td>80</td>
+      <td>80</td>
+    </tr>
+    <tr>
+      <td>9</td>
+      <td>90</td>
+      <td>90</td>
+      <td>90</td>
+    </tr>
+    <tr>
+      <td>10</td>
+      <td>100</td>
+      <td>100</td>
+      <td>100</td>
+    </tr>
+</table>
+
+If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery.
+
+# Java API
+
+Similar to [Spark's Java API](java-programming-guide.html), we also provide a Java API for Spark Streaming which allows all its features to be accessible from a Java program. This is defined in [spark.streaming.api.java] (api/streaming/index.html#spark.streaming.api.java.package) package and includes [JavaStreamingContext](api/streaming/index.html#spark.streaming.api.java.JavaStreamingContext) and [JavaDStream](api/streaming/index.html#spark.streaming.api.java.JavaDStream) classes that provide the same methods as their Scala counterparts, but take Java functions (that is, Function, and Function2) and return Java data and collection types. Some of the key points to note are:
+
+1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#spark.api.java.function.Function) and [Function2](api/core/index.html#spark.api.java.function.Function2)
+1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types.
+
+Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `<spark repo>/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java`
+
+The streaming context and the socket stream from input source is started by using a `JavaStreamingContext`, that has the same parameters and provides the same input streams as its Scala counterpart.
+
+{% highlight java %}
+JavaStreamingContext ssc = new JavaStreamingContext(mesosUrl, "NetworkWordCount", Seconds(1));
+JavaDStream<String> lines = ssc.socketTextStream(ip, port);
+{% endhighlight %}
+
+
+Then the `lines` are split into words by using the `flatMap` function and [FlatMapFunction](api/core/index.html#spark.api.java.function.FlatMapFunction).
+
+{% highlight java %}
+JavaDStream<String> words = lines.flatMap(
+  new FlatMapFunction<String, String>() {
+    @Override
+    public Iterable<String> call(String x) {
+      return Lists.newArrayList(x.split(" "));
+    }
+  });
+{% endhighlight %}
+
+The `words` is then mapped to a [JavaPairDStream](api/streaming/index.html#spark.streaming.api.java.JavaPairDStream) of `(word, 1)` pairs using `map` and [PairFunction](api/core/index.html#spark.api.java.function.PairFunction). This is  reduced by using `reduceByKey` and [Function2](api/core/index.html#spark.api.java.function.Function2).
+
+{% highlight java %}
+JavaPairDStream<String, Integer> wordCounts = words.map(
+  new PairFunction<String, String, Integer>() {
+    @Override
+    public Tuple2<String, Integer> call(String s) throws Exception {
+      return new Tuple2<String, Integer>(s, 1);
+    }
+  }).reduceByKey(
+  new Function2<Integer, Integer, Integer>() {
+    @Override
+    public Integer call(Integer i1, Integer i2) throws Exception {
+      return i1 + i2;
+    }
+  });
+{% endhighlight %}
+
+
+
+# Where to Go from Here
+* API docs - [Scala](api/streaming/index.html#spark.streaming.package) and [Java](api/streaming/index.html#spark.streaming.api.java.package)
+* More examples - [Scala](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/mesos/spark/tree/master/examples/src/main/java/spark/streaming/examples)
+* [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
diff --git a/docs/tuning.md b/docs/tuning.md
index 9aaa53cd65205901e62f39924b696abbecff4d8c..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])
@@ -213,10 +213,10 @@ but at a high level, managing how frequently full GC takes place can help in red
 
 Clusters will not be fully utilized unless you set the level of parallelism for each operation high
 enough. Spark automatically sets the number of "map" tasks to run on each file according to its size
-(though you can control it through optional parameters to `SparkContext.textFile`, etc), but for
-distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses a default value of 8.
-You can pass the level of parallelism as a second argument (see the
-[`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation),
+(though you can control it through optional parameters to `SparkContext.textFile`, etc), and for
+distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest
+parent RDD's number of partitions. You can pass the level of parallelism as a second argument
+(see the [`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation),
 or set the system property `spark.default.parallelism` to change the default.
 In general, we recommend 2-3 tasks per CPU core in your cluster.
 
@@ -233,7 +233,7 @@ number of cores in your clusters.
 
 ## Broadcasting Large Variables
 
-Using the [broadcast functionality](scala-programming-guide#broadcast-variables)
+Using the [broadcast functionality](scala-programming-guide.html#broadcast-variables)
 available in `SparkContext` can greatly reduce the size of each serialized task, and the cost
 of launching a job over a cluster. If your tasks use any large object from the driver program
 inside of them (e.g. a static lookup table), consider turning it into a broadcast variable.
diff --git a/ec2/README b/ec2/README
index 58dc087a811850fa782010a47d1bf7b285c95e84..0add81312c4285159a6f0b58f45b1180b97ac63f 100644
--- a/ec2/README
+++ b/ec2/README
@@ -1,4 +1,4 @@
 This folder contains a script, spark-ec2, for launching Spark clusters on
 Amazon EC2. Usage instructions are available online at:
 
-https://github.com/mesos/spark/wiki/Running-Spark-on-Amazon-EC2
+http://spark-project.org/docs/latest/ec2-scripts.html
diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
new file mode 100644
index 0000000000000000000000000000000000000000..166a884c889e4b154a4fe1b29c8aa61b4c382bed
--- /dev/null
+++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
@@ -0,0 +1,11 @@
+#!/bin/bash
+
+# These variables are automatically filled in by the mesos-ec2 script.
+export MESOS_MASTERS="{{master_list}}"
+export MESOS_SLAVES="{{slave_list}}"
+export MESOS_ZOO_LIST="{{zoo_list}}"
+export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}"
+export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}"
+export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}"
+export MODULES="{{modules}}"
+export SWAP_MB="{{swap}}"
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index a5384d3bda1c61d38cec2f4a6d2e70ff1fd84938..9f2daad2b632333b9dc288493d5ec52d908531db 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -35,7 +35,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType
 from boto import ec2
 
 # A static URL from which to figure out the latest Mesos EC2 AMI
-LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.6"
+LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.7"
 
 
 # Configure and parse our command-line arguments
@@ -82,12 +82,21 @@ def parse_args():
   parser.add_option("--spot-price", metavar="PRICE", type="float",
       help="If specified, launch slaves as spot instances with the given " +
             "maximum price (in dollars)")
-  parser.add_option("-c", "--cluster-type", default="mesos",
-      help="'mesos' for a mesos cluster, 'standalone' for a standalone spark cluster (default: mesos)")
+  parser.add_option("--cluster-type", type="choice", metavar="TYPE",
+      choices=["mesos", "standalone"], default="standalone",
+      help="'mesos' for a Mesos cluster, 'standalone' for a standalone " +
+           "Spark cluster (default: standalone)")
+  parser.add_option("--ganglia", action="store_true", default=True,
+      help="Setup Ganglia monitoring on cluster (default: on). NOTE: " +
+           "the Ganglia page will be publicly accessible")
+  parser.add_option("--no-ganglia", action="store_false", dest="ganglia",
+      help="Disable Ganglia monitoring for the cluster")
+  parser.add_option("--old-scripts", action="store_true", default=False,
+      help="Use old mesos-ec2 scripts, for Spark <= 0.6 AMIs")
   parser.add_option("-u", "--user", default="root",
-      help="The ssh user you want to connect as (default: root)")
+      help="The SSH user you want to connect as (default: root)")
   parser.add_option("--delete-groups", action="store_true", default=False,
-      help="When destroying a cluster, also destroy the security groups that were created")
+      help="When destroying a cluster, delete the security groups that were created")
             
   (opts, args) = parser.parse_args()
   if len(args) != 2:
@@ -164,22 +173,23 @@ def launch_cluster(conn, opts, cluster_name):
     master_group.authorize(src_group=zoo_group)
     master_group.authorize('tcp', 22, 22, '0.0.0.0/0')
     master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
+    master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0')
+    master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0')
+    master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0')
     if opts.cluster_type == "mesos":
-      master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0')
-      master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0')
-      master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0')
       master_group.authorize('tcp', 38090, 38090, '0.0.0.0/0')
+    if opts.ganglia:
+      master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0')
   if slave_group.rules == []: # Group was just now created
     slave_group.authorize(src_group=master_group)
     slave_group.authorize(src_group=slave_group)
     slave_group.authorize(src_group=zoo_group)
     slave_group.authorize('tcp', 22, 22, '0.0.0.0/0')
     slave_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
-    if opts.cluster_type == "mesos":
-      slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0')
-      slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0')
-      slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0')
-      slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0')
+    slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0')
+    slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0')
+    slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0')
+    slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0')
   if zoo_group.rules == []: # Group was just now created
     zoo_group.authorize(src_group=master_group)
     zoo_group.authorize(src_group=slave_group)
@@ -358,19 +368,38 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
 # Deploy configuration files and run setup scripts on a newly launched
 # or started EC2 cluster.
 def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_key):
-  print "Deploying files to master..."
-  deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, zoo_nodes)
   master = master_nodes[0].public_dns_name
   if deploy_ssh_key:
     print "Copying SSH key %s to master..." % opts.identity_file
     ssh(master, opts, 'mkdir -p ~/.ssh')
     scp(master, opts, opts.identity_file, '~/.ssh/id_rsa')
     ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa')
-  print "Running setup on master..."
+
   if opts.cluster_type == "mesos":
-    setup_mesos_cluster(master, opts)
+    modules = ['ephemeral-hdfs', 'persistent-hdfs', 'mesos']
   elif opts.cluster_type == "standalone":
-    setup_standalone_cluster(master, slave_nodes, opts)
+    modules = ['ephemeral-hdfs', 'persistent-hdfs', 'spark-standalone']
+
+  if opts.ganglia:
+    modules.append('ganglia')
+
+  if not opts.old_scripts:
+    # NOTE: We should clone the repository before running deploy_files to
+    # prevent ec2-variables.sh from being overwritten
+    ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git")
+
+  print "Deploying files to master..."
+  deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes,
+          zoo_nodes, modules)
+
+  print "Running setup on master..."
+  if opts.old_scripts:
+    if opts.cluster_type == "mesos":
+      setup_mesos_cluster(master, opts)
+    elif opts.cluster_type == "standalone":
+      setup_standalone_cluster(master, slave_nodes, opts)
+  else:
+    setup_spark_cluster(master, opts)
   print "Done!"
 
 def setup_mesos_cluster(master, opts):
@@ -383,6 +412,17 @@ def setup_standalone_cluster(master, slave_nodes, opts):
   ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips))
   ssh(master, opts, "/root/spark/bin/start-all.sh")
   
+def setup_spark_cluster(master, opts):
+  ssh(master, opts, "chmod u+x spark-ec2/setup.sh")
+  ssh(master, opts, "spark-ec2/setup.sh")
+  if opts.cluster_type == "mesos":
+    print "Mesos cluster started at http://%s:8080" % master
+  elif opts.cluster_type == "standalone":
+    print "Spark standalone cluster started at http://%s:8080" % master
+
+  if opts.ganglia:
+    print "Ganglia started at http://%s:5080/ganglia" % master
+
 
 # Wait for a whole cluster (masters, slaves and ZooKeeper) to start up
 def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes, zoo_nodes):
@@ -427,7 +467,8 @@ def get_num_disks(instance_type):
 # cluster (e.g. lists of masters and slaves). Files are only deployed to
 # the first master instance in the cluster, and we expect the setup
 # script to be run on that instance to copy them to other nodes.
-def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes):
+def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes,
+        modules):
   active_master = master_nodes[0].public_dns_name
 
   num_disks = get_num_disks(opts.instance_type)
@@ -459,7 +500,9 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes):
     "cluster_url": cluster_url,
     "hdfs_data_dirs": hdfs_data_dirs,
     "mapred_local_dirs": mapred_local_dirs,
-    "spark_local_dirs": spark_local_dirs
+    "spark_local_dirs": spark_local_dirs,
+    "swap": str(opts.swap),
+    "modules": '\n'.join(modules)
   }
 
   # Create a temp directory in which we will place all the files to be
@@ -497,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 3355deb6b77cd03006fe239a0ec991c82e875c6e..270777e29c02774476e439d7d80c3b2fcda981e6 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.0-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -19,7 +19,11 @@
       <groupId>org.eclipse.jetty</groupId>
       <artifactId>jetty-server</artifactId>
     </dependency>
-
+    <dependency>
+      <groupId>com.twitter</groupId>
+      <artifactId>algebird-core_2.9.2</artifactId>
+      <version>0.1.11</version>
+    </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.version}</artifactId>
@@ -45,11 +49,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-      <activation>
-        <property>
-          <name>!hadoopVersion</name>
-        </property>
-      </activation>
       <dependencies>
         <dependency>
           <groupId>org.spark-project</groupId>
@@ -57,6 +56,12 @@
           <version>${project.version}</version>
           <classifier>hadoop1</classifier>
         </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-streaming</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop1</classifier>
+        </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-core</artifactId>
@@ -77,12 +82,6 @@
     </profile>
     <profile>
       <id>hadoop2</id>
-      <activation>
-        <property>
-          <name>hadoopVersion</name>
-          <value>2</value>
-        </property>
-      </activation>
       <dependencies>
         <dependency>
           <groupId>org.spark-project</groupId>
@@ -90,6 +89,12 @@
           <version>${project.version}</version>
           <classifier>hadoop2</classifier>
         </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-streaming</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2</classifier>
+        </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-core</artifactId>
diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java
index 29839d5668cb7467e28f4a37c77a2fbc38f1a3e1..8b0a9b6808e8a96df560c817dd5582958fa3a207 100644
--- a/examples/src/main/java/spark/examples/JavaHdfsLR.java
+++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java
@@ -10,6 +10,9 @@ import java.util.Arrays;
 import java.util.StringTokenizer;
 import java.util.Random;
 
+/**
+ * Logistic regression based classification.
+ */
 public class JavaHdfsLR {
 
   static int D = 10;   // Number of dimensions
@@ -85,7 +88,8 @@ public class JavaHdfsLR {
       System.exit(1);
     }
 
-    JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR");
+    JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR",
+        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
     JavaRDD<String> lines = sc.textFile(args[1]);
     JavaRDD<DataPoint> points = lines.map(new ParsePoint()).cache();
     int ITERATIONS = Integer.parseInt(args[2]);
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/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java
index e3bd881b8f99ffdad8d61ff26e7ef1b512c63748..b319bdab44e77b413cbde5ee5dcde2f1161aac60 100644
--- a/examples/src/main/java/spark/examples/JavaTC.java
+++ b/examples/src/main/java/spark/examples/JavaTC.java
@@ -28,7 +28,7 @@ public class JavaTC {
       Tuple2<Integer, Integer> e = new Tuple2<Integer, Integer>(from, to);
       if (from != to) edges.add(e);
     }
-    return new ArrayList(edges);
+    return new ArrayList<Tuple2<Integer, Integer>>(edges);
   }
 
   static class ProjectFn extends PairFunction<Tuple2<Integer, Tuple2<Integer, Integer>>,
@@ -46,7 +46,8 @@ public class JavaTC {
       System.exit(1);
     }
 
-    JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC");
+    JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC",
+        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
     Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2;
     JavaPairRDD<Integer, Integer> tc = sc.parallelizePairs(generateGraph(), slices).cache();
 
diff --git a/examples/src/main/java/spark/examples/JavaWordCount.java b/examples/src/main/java/spark/examples/JavaWordCount.java
index a44cf8a1206b846dc0c493ac370604c3eb81913a..9d4c7a252df1c5d4b2b142f54cb877e921656f94 100644
--- a/examples/src/main/java/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/spark/examples/JavaWordCount.java
@@ -18,7 +18,8 @@ public class JavaWordCount {
       System.exit(1);
     }
 
-    JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount");
+    JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount",
+        System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
     JavaRDD<String> lines = ctx.textFile(args[1], 1);
 
     JavaRDD<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@@ -29,7 +30,7 @@ public class JavaWordCount {
     
     JavaPairRDD<String, Integer> ones = words.map(new PairFunction<String, String, Integer>() {
       public Tuple2<String, Integer> call(String s) {
-        return new Tuple2(s, 1);
+        return new Tuple2<String, Integer>(s, 1);
       }
     });
     
diff --git a/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
new file mode 100644
index 0000000000000000000000000000000000000000..e24c6ddaa79296ff98b5508597fa0d146981befa
--- /dev/null
+++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
@@ -0,0 +1,51 @@
+package spark.streaming.examples;
+
+import spark.api.java.function.Function;
+import spark.streaming.*;
+import spark.streaming.api.java.*;
+import spark.streaming.dstream.SparkFlumeEvent;
+
+/**
+ *  Produces a count of events received from Flume.
+ *
+ *  This should be used in conjunction with an AvroSink in Flume. It will start
+ *  an Avro server on at the request host:port address and listen for requests.
+ *  Your Flume AvroSink should be pointed to this address.
+ *
+ *  Usage: JavaFlumeEventCount <master> <host> <port>
+ *
+ *    <master> is a Spark master URL
+ *    <host> is the host the Flume receiver will be started on - a receiver
+ *           creates a server and listens for flume events.
+ *    <port> is the port the Flume receiver will listen on.
+ */
+public class JavaFlumeEventCount {
+  public static void main(String[] args) {
+    if (args.length != 3) {
+      System.err.println("Usage: JavaFlumeEventCount <master> <host> <port>");
+      System.exit(1);
+    }
+
+    String master = args[0];
+    String host = args[1];
+    int port = Integer.parseInt(args[2]);
+
+    Duration batchInterval = new Duration(2000);
+
+    JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
+            System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+
+    JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
+
+    flumeStream.count();
+
+    flumeStream.count().map(new Function<Long, String>() {
+      @Override
+      public String call(Long in) {
+        return "Received " + in + " flume events.";
+      }
+    }).print();
+
+    sc.start();
+  }
+}
diff --git a/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
new file mode 100644
index 0000000000000000000000000000000000000000..3e57580fd4e6c4613dbc30d274824f5e052ad9c7
--- /dev/null
+++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
@@ -0,0 +1,62 @@
+package spark.streaming.examples;
+
+import com.google.common.collect.Lists;
+import scala.Tuple2;
+import spark.api.java.function.FlatMapFunction;
+import spark.api.java.function.Function2;
+import spark.api.java.function.PairFunction;
+import spark.streaming.Duration;
+import spark.streaming.api.java.JavaDStream;
+import spark.streaming.api.java.JavaPairDStream;
+import spark.streaming.api.java.JavaStreamingContext;
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
+ * Usage: NetworkWordCount <master> <hostname> <port>
+ *   <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
+ *   <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ *    `$ nc -lk 9999`
+ * and then run the example
+ *    `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999`
+ */
+public class JavaNetworkWordCount {
+  public static void main(String[] args) {
+    if (args.length < 3) {
+      System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
+          "In local mode, <master> should be 'local[n]' with n > 1");
+      System.exit(1);
+    }
+
+    // Create the context with a 1 second batch size
+    JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount",
+            new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+
+    // Create a NetworkInputDStream on target ip:port and count the
+    // words in input stream of \n delimited test (eg. generated by 'nc')
+    JavaDStream<String> lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2]));
+    JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
+      @Override
+      public Iterable<String> call(String x) {
+        return Lists.newArrayList(x.split(" "));
+      }
+    });
+    JavaPairDStream<String, Integer> wordCounts = words.map(
+      new PairFunction<String, String, Integer>() {
+        @Override
+        public Tuple2<String, Integer> call(String s) throws Exception {
+          return new Tuple2<String, Integer>(s, 1);
+        }
+      }).reduceByKey(new Function2<Integer, Integer, Integer>() {
+        @Override
+        public Integer call(Integer i1, Integer i2) throws Exception {
+          return i1 + i2;
+        }
+      });
+
+    wordCounts.print();
+    ssc.start();
+
+  }
+}
diff --git a/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
new file mode 100644
index 0000000000000000000000000000000000000000..15b82c8da15946d486147e3d541df5bd1d6a9595
--- /dev/null
+++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
@@ -0,0 +1,63 @@
+package spark.streaming.examples;
+
+import com.google.common.collect.Lists;
+import scala.Tuple2;
+import spark.api.java.JavaRDD;
+import spark.api.java.function.Function2;
+import spark.api.java.function.PairFunction;
+import spark.streaming.Duration;
+import spark.streaming.api.java.JavaDStream;
+import spark.streaming.api.java.JavaPairDStream;
+import spark.streaming.api.java.JavaStreamingContext;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+public class JavaQueueStream {
+  public static void main(String[] args) throws InterruptedException {
+    if (args.length < 1) {
+      System.err.println("Usage: JavaQueueStream <master>");
+      System.exit(1);
+    }
+
+    // Create the context
+    JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000),
+            System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+
+    // Create the queue through which RDDs can be pushed to
+    // a QueueInputDStream
+    Queue<JavaRDD<Integer>> rddQueue = new LinkedList<JavaRDD<Integer>>();
+
+    // Create and push some RDDs into the queue
+    List<Integer> list = Lists.newArrayList();
+    for (int i = 0; i < 1000; i++) {
+      list.add(i);
+    }
+
+    for (int i = 0; i < 30; i++) {
+      rddQueue.add(ssc.sc().parallelize(list));
+    }
+
+
+    // Create the QueueInputDStream and use it do some processing
+    JavaDStream<Integer> inputStream = ssc.queueStream(rddQueue);
+    JavaPairDStream<Integer, Integer> mappedStream = inputStream.map(
+        new PairFunction<Integer, Integer, Integer>() {
+          @Override
+          public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+            return new Tuple2<Integer, Integer>(i % 10, 1);
+          }
+        });
+    JavaPairDStream<Integer, Integer> reducedStream = mappedStream.reduceByKey(
+      new Function2<Integer, Integer, Integer>() {
+        @Override
+        public Integer call(Integer i1, Integer i2) throws Exception {
+          return i1 + i2;
+        }
+    });
+
+    reducedStream.print();
+    ssc.start();
+  }
+}
diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala
index 230097c7db2a762b71966ba3212d47eea3ab0ba1..ba59be1687235710741b72185097d0d33be7bf8c 100644
--- a/examples/src/main/scala/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala
@@ -9,19 +9,21 @@ object BroadcastTest {
       System.exit(1)
     }  
     
-    val spark = new SparkContext(args(0), "Broadcast Test")
+    val sc = new SparkContext(args(0), "Broadcast Test",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val slices = if (args.length > 1) args(1).toInt else 2
     val num = if (args.length > 2) args(2).toInt else 1000000
 
     var arr1 = new Array[Int](num)
-    for (i <- 0 until arr1.length) 
+    for (i <- 0 until arr1.length) {
       arr1(i) = i
+    }
     
     for (i <- 0 until 2) {
       println("Iteration " + i)
       println("===========")
-      val barr1 = spark.broadcast(arr1)
-      spark.parallelize(1 to 10, slices).foreach {
+      val barr1 = sc.broadcast(arr1)
+      sc.parallelize(1 to 10, slices).foreach {
         i => println(barr1.value.size)
       }
     }
diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
index c89f3dac0c5a396b74039b909213025a847a2c7c..21a90f2e5ad5b53aba3976568dd8b0eb47dfc500 100644
--- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
@@ -9,9 +9,10 @@ object ExceptionHandlingTest {
       System.exit(1)
     }
 
-    val sc = new SparkContext(args(0), "ExceptionHandlingTest")
+    val sc = new SparkContext(args(0), "ExceptionHandlingTest",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     sc.parallelize(0 until sc.defaultParallelism).foreach { i =>
-      if (Math.random > 0.75)
+      if (math.random > 0.75)
         throw new Exception("Testing exception handling")
     }
 
diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala
index 86dfba3a404987d9aa53f8829b5e22886264cb3c..a6603653f1503044ba41a82508067d64e70ccac1 100644
--- a/examples/src/main/scala/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/GroupByTest.scala
@@ -9,14 +9,15 @@ object GroupByTest {
     if (args.length == 0) {
       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
     var numReducers = if (args.length > 4) args(4).toInt else numMappers
 
-    val sc = new SparkContext(args(0), "GroupBy Test")
+    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/HdfsTest.scala b/examples/src/main/scala/spark/examples/HdfsTest.scala
index 7a4530609d737c44e0fdbbbdade49581ce08973e..dd61c467f7fc7b3fabc6f319c15ac7adffe18dd3 100644
--- a/examples/src/main/scala/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/spark/examples/HdfsTest.scala
@@ -4,7 +4,8 @@ import spark._
 
 object HdfsTest {
   def main(args: Array[String]) {
-    val sc = new SparkContext(args(0), "HdfsTest")
+    val sc = new SparkContext(args(0), "HdfsTest",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val file = sc.textFile(args(1))
     val mapped = file.map(s => s.length).cache()
     for (iter <- 1 to 10) {
diff --git a/examples/src/main/scala/spark/examples/LocalALS.scala b/examples/src/main/scala/spark/examples/LocalALS.scala
index 10e03359c9596a49c5ebaf29b3e735e85c0394b7..2de810e062b787b15174fb83c108f0aca33aa23e 100644
--- a/examples/src/main/scala/spark/examples/LocalALS.scala
+++ b/examples/src/main/scala/spark/examples/LocalALS.scala
@@ -1,11 +1,13 @@
 package spark.examples
 
-import java.util.Random
 import scala.math.sqrt
 import cern.jet.math._
 import cern.colt.matrix._
 import cern.colt.matrix.linalg._
 
+/**
+ * Alternating least squares matrix factorization.
+ */
 object LocalALS {
   // Parameters set through command line arguments
   var M = 0 // Number of movies
diff --git a/examples/src/main/scala/spark/examples/LocalKMeans.scala b/examples/src/main/scala/spark/examples/LocalKMeans.scala
index b442c604cdd2c975a2b535768d5ba382004190c6..4849f216fb2933faa17542897a584a70d88c2fda 100644
--- a/examples/src/main/scala/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala
@@ -6,74 +6,77 @@ import spark.SparkContext._
 import scala.collection.mutable.HashMap
 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/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala
index 9553162004985bc133a01327c4f877835606152b..cd73f553d6e23509008974d272822ac18227e459 100644
--- a/examples/src/main/scala/spark/examples/LocalLR.scala
+++ b/examples/src/main/scala/spark/examples/LocalLR.scala
@@ -3,6 +3,9 @@ package spark.examples
 import java.util.Random
 import spark.util.Vector
 
+/**
+ * Logistic regression based classification.
+ */
 object LocalLR {
   val N = 10000  // Number of data points
   val D = 10   // Number of dimensions
diff --git a/examples/src/main/scala/spark/examples/LogQuery.scala b/examples/src/main/scala/spark/examples/LogQuery.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6497596d35f34a81b07f4cbdf972d0eabcabf032
--- /dev/null
+++ b/examples/src/main/scala/spark/examples/LogQuery.scala
@@ -0,0 +1,68 @@
+package spark.examples
+
+import spark.SparkContext
+import spark.SparkContext._
+/**
+ * Executes a roll up-style query against Apache logs.
+ */
+object LogQuery {
+  val exampleApacheLogs = List(
+    """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""".stripMargin.replace("\n", ""),
+    """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""".stripMargin.replace("\n", "")
+  )
+
+  def main(args: Array[String]) {
+    if (args.length == 0) {
+      System.err.println("Usage: LogQuery <master> [logFile]")
+      System.exit(1)
+    }
+
+    val sc = new SparkContext(args(0), "Log Query",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    val dataSet =
+      if (args.length == 2) sc.textFile(args(1))
+      else sc.parallelize(exampleApacheLogs)
+
+    val apacheLogRegex =
+      """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r
+
+    /** Tracks the total query count and number of aggregate bytes for a particular group. */
+    class Stats(val count: Int, val numBytes: Int) extends Serializable {
+      def merge(other: Stats) = new Stats(count + other.count, numBytes + other.numBytes)
+      override def toString = "bytes=%s\tn=%s".format(numBytes, count)
+    }
+
+    def extractKey(line: String): (String, String, String) = {
+      apacheLogRegex.findFirstIn(line) match {
+        case Some(apacheLogRegex(ip, _, user, dateTime, query, status, bytes, referer, ua)) =>
+          if (user != "\"-\"") (ip, user, query)
+          else (null, null, null)
+        case _ => (null, null, null)
+      }
+    }
+
+    def extractStats(line: String): Stats = {
+      apacheLogRegex.findFirstIn(line) match {
+        case Some(apacheLogRegex(ip, _, user, dateTime, query, status, bytes, referer, ua)) =>
+          new Stats(1, bytes.toInt)
+        case _ => new Stats(1, 0)
+      }
+    }
+
+    dataSet.map(line => (extractKey(line), extractStats(line)))
+      .reduceByKey((a, b) => a.merge(b))
+      .collect().foreach{
+        case (user, query) => println("%s\t%s".format(user, query))}
+  }
+}
diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
index 83ae014e9458063c41a15e861c1ee638b43f8ca4..a0aaf609186b74813f010ea419465102ecdfd0d0 100644
--- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
@@ -8,25 +8,29 @@ object MultiBroadcastTest {
       System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
       System.exit(1)
     }
-    
-    val spark = new SparkContext(args(0), "Broadcast Test")
+
+    val sc = new SparkContext(args(0), "Broadcast Test",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
     val slices = if (args.length > 1) args(1).toInt else 2
     val num = if (args.length > 2) args(2).toInt else 1000000
 
     var arr1 = new Array[Int](num)
-    for (i <- 0 until arr1.length) 
+    for (i <- 0 until arr1.length) {
       arr1(i) = i
-    
+    }
+
     var arr2 = new Array[Int](num)
-    for (i <- 0 until arr2.length)
+    for (i <- 0 until arr2.length) {
       arr2(i) = i
+    }
 
-    val barr1 = spark.broadcast(arr1)
-    val barr2 = spark.broadcast(arr2)
-    spark.parallelize(1 to 10, slices).foreach {
+    val barr1 = sc.broadcast(arr1)
+    val barr2 = sc.broadcast(arr2)
+    sc.parallelize(1 to 10, slices).foreach {
       i => println(barr1.value.size + barr2.value.size)
     }
-    
+
     System.exit(0)
   }
 }
diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
index 50b3a263b4feb993dbf5b5783758b0d75aff10ec..461b84a2c66232d7a2c01e31b97b9ded377909a2 100644
--- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
@@ -11,15 +11,16 @@ 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
     var numReducers = if (args.length > 4) args(4).toInt else numMappers
     var ratio = if (args.length > 5) args(5).toInt else 5.0
 
-    val sc = new SparkContext(args(0), "GroupBy Test")
-    
+    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 d2117a263e6d8269d13619b134ed8962fd4014b7..435675f9de489d65988fded440e91084a820cdf0 100644
--- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
@@ -10,19 +10,20 @@ 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
     var numReducers = if (args.length > 4) args(4).toInt else numMappers
 
-    val sc = new SparkContext(args(0), "GroupBy Test")
-    
+    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
 
       // map output sizes lineraly increase from the 1st to the last
-      numKVPairs = (1. * (p + 1) / numMappers * numKVPairs).toInt
+      numKVPairs = (1.0 * (p + 1) / numMappers * numKVPairs).toInt
 
       var arr1 = new Array[(Int, Array[Byte])](numKVPairs)
       for (i <- 0 until numKVPairs) {
@@ -31,11 +32,11 @@ object SkewedGroupByTest {
         arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr)
       }
       arr1
-    }.cache
+    }.cache()
     // Enforce that everything has been calculated and in cache
-    pairs1.count
+    pairs1.count()
     
-    println(pairs1.groupByKey(numReducers).count)
+    println(pairs1.groupByKey(numReducers).count())
 
     System.exit(0)
   }
diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala
index fb28e2c93273f74603de6e8e2e4688dec42c9c26..8fb3b0fb2ad78eadbb906c8e5c11cc4131b8b929 100644
--- a/examples/src/main/scala/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/spark/examples/SparkALS.scala
@@ -1,13 +1,14 @@
 package spark.examples
 
-import java.io.Serializable
-import java.util.Random
 import scala.math.sqrt
 import cern.jet.math._
 import cern.colt.matrix._
 import cern.colt.matrix.linalg._
 import spark._
 
+/**
+ * Alternating least squares matrix factorization.
+ */
 object SparkALS {
   // Parameters set through command line arguments
   var M = 0 // Number of movies
@@ -42,7 +43,7 @@ object SparkALS {
     return sqrt(sumSqs / (M * U))
   }
 
-  def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
+  def update(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D],
     R: DoubleMatrix2D) : DoubleMatrix1D =
   {
     val U = us.size
@@ -68,51 +69,33 @@ object SparkALS {
     return solved2D.viewColumn(0)
   }
 
-  def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D],
-    R: DoubleMatrix2D) : DoubleMatrix1D =
-  {
-    val M = ms.size
-    val F = ms(0).size
-    val XtX = factory2D.make(F, F)
-    val Xty = factory1D.make(F)
-    // For each movie that the user rated
-    for (i <- 0 until M) {
-      val m = ms(i)
-      // Add m * m^t to XtX
-      blas.dger(1, m, m, XtX)
-      // Add m * rating to Xty
-      blas.daxpy(R.get(i, j), m, Xty)
-    }
-    // Add regularization coefs to diagonal terms
-    for (d <- 0 until F) {
-      XtX.set(d, d, XtX.get(d, d) + LAMBDA * M)
+  def main(args: Array[String]) {
+    if (args.length == 0) {
+      System.err.println("Usage: SparkALS <master> [<M> <U> <F> <iters> <slices>]")
+      System.exit(1)
     }
-    // Solve it with Cholesky
-    val ch = new CholeskyDecomposition(XtX)
-    val Xty2D = factory2D.make(Xty.toArray, F)
-    val solved2D = ch.solve(Xty2D)
-    return solved2D.viewColumn(0)
-  }
 
-  def main(args: Array[String]) {
     var host = ""
     var slices = 0
-    args match {
-      case Array(m, u, f, iters, slices_, host_) => {
-        M = m.toInt
-        U = u.toInt
-        F = f.toInt
-        ITERATIONS = iters.toInt
-        slices = slices_.toInt
-        host = host_
-      }
-      case _ => {
-        System.err.println("Usage: SparkALS <M> <U> <F> <iters> <slices> <master>")
+
+    val options = (0 to 5).map(i => if (i < args.length) Some(args(i)) else None)
+
+    options.toArray match {
+      case Array(host_, m, u, f, iters, slices_) =>
+        host = host_.get
+        M = m.getOrElse("100").toInt
+        U = u.getOrElse("500").toInt
+        F = f.getOrElse("10").toInt
+        ITERATIONS = iters.getOrElse("5").toInt
+        slices = slices_.getOrElse("2").toInt
+      case _ =>
+        System.err.println("Usage: SparkALS <master> [<M> <U> <F> <iters> <slices>]")
         System.exit(1)
-      }
     }
-    printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS);
-    val spark = new SparkContext(host, "SparkALS")
+    printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
+
+    val sc = new SparkContext(host, "SparkALS",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     
     val R = generateR()
 
@@ -121,19 +104,19 @@ object SparkALS {
     var us = Array.fill(U)(factory1D.random(F))
 
     // Iteratively update movies then users
-    val Rc  = spark.broadcast(R)
-    var msc = spark.broadcast(ms)
-    var usc = spark.broadcast(us)
+    val Rc  = sc.broadcast(R)
+    var msb = sc.broadcast(ms)
+    var usb = sc.broadcast(us)
     for (iter <- 1 to ITERATIONS) {
       println("Iteration " + iter + ":")
-      ms = spark.parallelize(0 until M, slices)
-                .map(i => updateMovie(i, msc.value(i), usc.value, Rc.value))
+      ms = sc.parallelize(0 until M, slices)
+                .map(i => update(i, msb.value(i), usb.value, Rc.value))
                 .toArray
-      msc = spark.broadcast(ms) // Re-broadcast ms because it was updated
-      us = spark.parallelize(0 until U, slices)
-                .map(i => updateUser(i, usc.value(i), msc.value, Rc.value))
+      msb = sc.broadcast(ms) // Re-broadcast ms because it was updated
+      us = sc.parallelize(0 until U, slices)
+                .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value)))
                 .toArray
-      usc = spark.broadcast(us) // Re-broadcast us because it was updated
+      usb = sc.broadcast(us) // Re-broadcast us because it was updated
       println("RMSE = " + rmse(R, ms, us))
       println()
     }
diff --git a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
index 5b2bc84d6900fa8099ef065091935bb7ef20f15d..0f42f405a058cf9f2a218004b47c66d2330810d6 100644
--- a/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/spark/examples/SparkHdfsLR.scala
@@ -5,6 +5,9 @@ import scala.math.exp
 import spark.util.Vector
 import spark._
 
+/**
+ * Logistic regression based classification.
+ */
 object SparkHdfsLR {
   val D = 10   // Numer of dimensions
   val rand = new Random(42)
@@ -29,7 +32,8 @@ object SparkHdfsLR {
       System.err.println("Usage: SparkHdfsLR <master> <file> <iters>")
       System.exit(1)
     }
-    val sc = new SparkContext(args(0), "SparkHdfsLR")
+    val sc = new SparkContext(args(0), "SparkHdfsLR",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val lines = sc.textFile(args(1))
     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 63759613906b46a457025adcad6f9146dc01de42..4161c59fead2046851428f799f1ecbc07b1eedf8 100644
--- a/examples/src/main/scala/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala
@@ -7,6 +7,9 @@ import spark.SparkContext._
 import scala.collection.mutable.HashMap
 import scala.collection.mutable.HashSet
 
+/**
+ * K-means clustering.
+ */
 object SparkKMeans {
   val R = 1000     // Scaling factor
   val rand = new Random(42)
@@ -36,7 +39,8 @@ object SparkKMeans {
         System.err.println("Usage: SparkLocalKMeans <master> <file> <k> <convergeDist>")
         System.exit(1)
     }
-    val sc = new SparkContext(args(0), "SparkLocalKMeans")
+    val sc = new SparkContext(args(0), "SparkLocalKMeans",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val lines = sc.textFile(args(1))
     val data = lines.map(parseVector _).cache()
     val K = args(2).toInt
@@ -60,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/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala
index aaaf062c8f9ea79a4a67b9314ba17eeebc9cc691..2f41aeb376fda07e6247c9a915050c3093301b13 100644
--- a/examples/src/main/scala/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/spark/examples/SparkLR.scala
@@ -5,6 +5,9 @@ import scala.math.exp
 import spark.util.Vector
 import spark._
 
+/**
+ * Logistic regression based classification.
+ */
 object SparkLR {
   val N = 10000  // Number of data points
   val D = 10   // Numer of dimensions
@@ -28,7 +31,8 @@ object SparkLR {
       System.err.println("Usage: SparkLR <master> [<slices>]")
       System.exit(1)
     }
-    val sc = new SparkContext(args(0), "SparkLR")
+    val sc = new SparkContext(args(0), "SparkLR",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val numSlices = if (args.length > 1) args(1).toInt else 2
     val points = sc.parallelize(generateData, numSlices).cache()
 
diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala
index 2f226f13805c02ef6c9f8bd6d73d3434283a022c..f598d2ff9c7cdf0565594eb36965fbdea5b27b25 100644
--- a/examples/src/main/scala/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/spark/examples/SparkPi.scala
@@ -4,13 +4,15 @@ import scala.math.random
 import spark._
 import SparkContext._
 
+/** Computes an approximation to pi */
 object SparkPi {
   def main(args: Array[String]) {
     if (args.length == 0) {
       System.err.println("Usage: SparkPi <master> [<slices>]")
       System.exit(1)
     }
-    val spark = new SparkContext(args(0), "SparkPi")
+    val spark = new SparkContext(args(0), "SparkPi",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val slices = if (args.length > 1) args(1).toInt else 2
     val n = 100000 * slices
     val count = spark.parallelize(1 to n, slices).map { i =>
diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala
index 90bae011adfb6381bf8fe2409d04f8a3a5598882..911ae8f168d03dbc46d792061a40875723f94982 100644
--- a/examples/src/main/scala/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/spark/examples/SparkTC.scala
@@ -9,7 +9,6 @@ import scala.collection.mutable
  * Transitive closure on a graph.
  */
 object SparkTC {
-
   val numEdges = 200
   val numVertices = 100
   val rand = new Random(42)
@@ -29,7 +28,8 @@ object SparkTC {
       System.err.println("Usage: SparkTC <master> [<slices>]")
       System.exit(1)
     }
-    val spark = new SparkContext(args(0), "SparkTC")
+    val spark = new SparkContext(args(0), "SparkTC",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     val slices = if (args.length > 1) args(1).toInt else 2
     var tc = spark.parallelize(generateGraph, slices).cache()
 
diff --git a/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3b847fe60340968d55ca6f1d1a2155f61e750bb1
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/ActorWordCount.scala
@@ -0,0 +1,158 @@
+package spark.streaming.examples
+
+import scala.collection.mutable.LinkedList
+import scala.util.Random
+
+import akka.actor.Actor
+import akka.actor.ActorRef
+import akka.actor.Props
+import akka.actor.actorRef2Scala
+
+import spark.streaming.Seconds
+import spark.streaming.StreamingContext
+import spark.streaming.StreamingContext.toPairDStreamFunctions
+import spark.streaming.receivers.Receiver
+import spark.util.AkkaUtils
+
+case class SubscribeReceiver(receiverActor: ActorRef)
+case class UnsubscribeReceiver(receiverActor: ActorRef)
+
+/**
+ * Sends the random content to every receiver subscribed with 1/2
+ *  second delay.
+ */
+class FeederActor extends Actor {
+
+  val rand = new Random()
+  var receivers: LinkedList[ActorRef] = new LinkedList[ActorRef]()
+
+  val strings: Array[String] = Array("words ", "may ", "count ")
+
+  def makeMessage(): String = {
+    val x = rand.nextInt(3)
+    strings(x) + strings(2 - x)
+  }
+
+  /*
+   * A thread to generate random messages
+   */
+  new Thread() {
+    override def run() {
+      while (true) {
+        Thread.sleep(500)
+        receivers.foreach(_ ! makeMessage)
+      }
+    }
+  }.start()
+
+  def receive: Receive = {
+
+    case SubscribeReceiver(receiverActor: ActorRef) =>
+      println("received subscribe from %s".format(receiverActor.toString))
+    receivers = LinkedList(receiverActor) ++ receivers
+
+    case UnsubscribeReceiver(receiverActor: ActorRef) =>
+      println("received unsubscribe from %s".format(receiverActor.toString))
+    receivers = receivers.dropWhile(x => x eq receiverActor)
+
+  }
+}
+
+/**
+ * A sample actor as receiver, is also simplest. This receiver actor
+ * goes and subscribe to a typical publisher/feeder actor and receives
+ * data.
+ *
+ * @see [[spark.streaming.examples.FeederActor]]
+ */
+class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String)
+extends Actor with Receiver {
+
+  lazy private val remotePublisher = context.actorFor(urlOfPublisher)
+
+  override def preStart = remotePublisher ! SubscribeReceiver(context.self)
+
+  def receive = {
+    case msg ⇒ context.parent ! pushBlock(msg.asInstanceOf[T])
+  }
+
+  override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self)
+
+}
+
+/**
+ * A sample feeder actor
+ *
+ * Usage: FeederActor <hostname> <port>
+ *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder would start on.
+ */
+object FeederActor {
+
+  def main(args: Array[String]) {
+    if(args.length < 2){
+      System.err.println(
+        "Usage: FeederActor <hostname> <port>\n"
+      )
+      System.exit(1)
+    }
+    val Seq(host, port) = args.toSeq
+
+
+    val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1
+    val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor")
+
+    println("Feeder started as:" + feeder)
+
+    actorSystem.awaitTermination();
+  }
+}
+
+/**
+ * A sample word count program demonstrating the use of plugging in
+ * Actor as Receiver
+ * Usage: ActorWordCount <master> <hostname> <port>
+ *   <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
+ *   <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
+ *
+ * To run this example locally, you may run Feeder Actor as
+ *    `$ ./run spark.streaming.examples.FeederActor 127.0.1.1 9999`
+ * and then run the example
+ *    `$ ./run spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
+ */
+object ActorWordCount {
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println(
+        "Usage: ActorWordCount <master> <hostname> <port>" +
+        "In local mode, <master> should be 'local[n]' with n > 1")
+      System.exit(1)
+    }
+
+    val Seq(master, host, port) = args.toSeq
+
+    // Create the context and set the batch size
+    val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    /*
+     * Following is the use of actorStream to plug in custom actor as receiver
+     *
+     * An important point to note:
+     * Since Actor may exist outside the spark framework, It is thus user's responsibility
+     * to ensure the type safety, i.e type of data received and InputDstream
+     * should be same.
+     *
+     * For example: Both actorStream and SampleActorReceiver are parameterized
+     * to same type to ensure type safety.
+     */
+
+    val lines = ssc.actorStream[String](
+      Props(new SampleActorReceiver[String]("akka://test@%s:%s/user/FeederActor".format(
+        host, port.toInt))), "SampleReceiver")
+
+    //compute wordcount
+    lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print()
+
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..39c76fd98a11e3ff4b5da2f98b4b8839a1a33f9b
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
@@ -0,0 +1,44 @@
+package spark.streaming.examples
+
+import spark.util.IntParam
+import spark.storage.StorageLevel
+import spark.streaming._
+
+/**
+ *  Produces a count of events received from Flume.
+ *
+ *  This should be used in conjunction with an AvroSink in Flume. It will start
+ *  an Avro server on at the request host:port address and listen for requests.
+ *  Your Flume AvroSink should be pointed to this address.
+ *
+ *  Usage: FlumeEventCount <master> <host> <port>
+ *
+ *    <master> is a Spark master URL
+ *    <host> is the host the Flume receiver will be started on - a receiver
+ *           creates a server and listens for flume events.
+ *    <port> is the port the Flume receiver will listen on.
+ */
+object FlumeEventCount {
+  def main(args: Array[String]) {
+    if (args.length != 3) {
+      System.err.println(
+        "Usage: FlumeEventCount <master> <host> <port>")
+      System.exit(1)
+    }
+
+    val Array(master, host, IntParam(port)) = args
+
+    val batchInterval = Milliseconds(2000)
+    // Create the context and set the batch size
+    val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval,
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    // Create a flume stream
+    val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)
+
+    // Print out the count of events received from this server in each batch
+    stream.count().map(cnt => "Received " + cnt + " flume events." ).print()
+
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9389f8a38dcca41a5a6d55efe9094bb68d8b98ee
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
@@ -0,0 +1,37 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+
+
+/**
+ * Counts words in new text files created in the given directory
+ * Usage: HdfsWordCount <master> <directory>
+ *   <master> is the Spark master URL.
+ *   <directory> is the directory that Spark Streaming will use to find and read new text files.
+ *
+ * To run this on your local machine on directory `localdir`, run this example
+ *    `$ ./run spark.streaming.examples.HdfsWordCount local[2] localdir`
+ * Then create a text file in `localdir` and the words in the file will get counted.
+ */
+object HdfsWordCount {
+  def main(args: Array[String]) {
+    if (args.length < 2) {
+      System.err.println("Usage: HdfsWordCount <master> <directory>")
+      System.exit(1)
+    }
+
+    // Create the context
+    val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    // Create the FileInputDStream on the directory and use the
+    // stream to count words in new files created
+    val lines = ssc.textFileStream(args(1))
+    val words = lines.flatMap(_.split(" "))
+    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+    wordCounts.print()
+    ssc.start()
+  }
+}
+
diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c3a9e491ba85406dd99262dfe1a82f6a1c42198b
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
@@ -0,0 +1,81 @@
+package spark.streaming.examples
+
+import java.util.Properties
+import kafka.message.Message
+import kafka.producer.SyncProducerConfig
+import kafka.producer._
+import spark.SparkContext
+import spark.streaming._
+import spark.streaming.StreamingContext._
+import spark.storage.StorageLevel
+import spark.streaming.util.RawTextHelper._
+
+/**
+ * Consumes messages from one or more topics in Kafka and does wordcount.
+ * Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>
+ *   <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
+ *   <zkQuorum> is a list of one or more zookeeper servers that make quorum
+ *   <group> is the name of kafka consumer group
+ *   <topics> is a list of one or more kafka topics to consume from
+ *   <numThreads> is the number of threads the kafka consumer should use
+ *
+ * Example:
+ *    `./run spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
+ */
+object KafkaWordCount {
+  def main(args: Array[String]) {
+    
+    if (args.length < 5) {
+      System.err.println("Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>")
+      System.exit(1)
+    }
+
+    val Array(master, zkQuorum, group, topics, numThreads) = args
+
+    val ssc =  new StreamingContext(master, "KafkaWordCount", Seconds(2),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+    ssc.checkpoint("checkpoint")
+
+    val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
+    val lines = ssc.kafkaStream[String](zkQuorum, group, topicpMap)
+    val words = lines.flatMap(_.split(" "))
+    val wordCounts = words.map(x => (x, 1l)).reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
+    wordCounts.print()
+    
+    ssc.start()
+  }
+}
+
+// Produces some random words between 1 and 100.
+object KafkaWordCountProducer {
+
+  def main(args: Array[String]) {
+    if (args.length < 2) {
+      System.err.println("Usage: KafkaWordCountProducer <zkQuorum> <topic> <messagesPerSec> <wordsPerMessage>")
+      System.exit(1)
+    }
+
+    val Array(zkQuorum, topic, messagesPerSec, wordsPerMessage) = args
+
+    // Zookeper connection properties
+    val props = new Properties()
+    props.put("zk.connect", zkQuorum)
+    props.put("serializer.class", "kafka.serializer.StringEncoder")
+    
+    val config = new ProducerConfig(props)
+    val producer = new Producer[String, String](config)
+
+    // Send some messages
+    while(true) {
+      val messages = (1 to messagesPerSec.toInt).map { messageNum =>
+        (1 to wordsPerMessage.toInt).map(x => scala.util.Random.nextInt(10).toString).mkString(" ")
+      }.toArray
+      println(messages.mkString(","))
+      val data = new ProducerData[String, String](topic, messages)
+      producer.send(data)
+      Thread.sleep(100)
+    }
+  }
+
+}
+
diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..704540c2bf51b35cad2647a31cc134bd82b0226c
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
@@ -0,0 +1,37 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+
+/**
+ * Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
+ * Usage: NetworkWordCount <master> <hostname> <port>
+ *   <master> is the Spark master URL. In local mode, <master> should be 'local[n]' with n > 1.
+ *   <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive data.
+ *
+ * To run this on your local machine, you need to first run a Netcat server
+ *    `$ nc -lk 9999`
+ * and then run the example
+ *    `$ ./run spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ */
+object NetworkWordCount {
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
+        "In local mode, <master> should be 'local[n]' with n > 1")
+      System.exit(1)
+    }
+
+    // Create the context with a 1 second batch size
+    val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    // Create a NetworkInputDStream on target ip:port and count the
+    // words in input stream of \n delimited test (eg. generated by 'nc') 
+    val lines = ssc.socketTextStream(args(1), args(2).toInt)
+    val words = lines.flatMap(_.split(" "))
+    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+    wordCounts.print()
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f450e2104018bbcc6a90b064b732b204babd2e40
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.examples
+
+import spark.RDD
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+
+import scala.collection.mutable.SynchronizedQueue
+
+object QueueStream {
+  
+  def main(args: Array[String]) {
+    if (args.length < 1) {
+      System.err.println("Usage: QueueStream <master>")
+      System.exit(1)
+    }
+    
+    // Create the context
+    val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    // Create the queue through which RDDs can be pushed to 
+    // a QueueInputDStream
+    val rddQueue = new SynchronizedQueue[RDD[Int]]()
+    
+    // Create the QueueInputDStream and use it do some processing
+    val inputStream = ssc.queueStream(rddQueue)
+    val mappedStream = inputStream.map(x => (x % 10, 1))
+    val reducedStream = mappedStream.reduceByKey(_ + _)
+    reducedStream.print()    
+    ssc.start()
+    
+    // Create and push some RDDs into
+    for (i <- 1 to 30) {
+      rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10)
+      Thread.sleep(1000)
+    }
+    ssc.stop()
+    System.exit(0)
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
new file mode 100644
index 0000000000000000000000000000000000000000..175281e0956d4c4b9d43f1ae811dae125b71c2f2
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
@@ -0,0 +1,47 @@
+package spark.streaming.examples
+
+import spark.util.IntParam
+import spark.storage.StorageLevel
+
+import spark.streaming._
+import spark.streaming.util.RawTextHelper
+
+/**
+ * Receives text from multiple rawNetworkStreams and counts how many '\n' delimited
+ * lines have the word 'the' in them. This is useful for benchmarking purposes. This
+ * will only work with spark.streaming.util.RawTextSender running on all worker nodes
+ * and with Spark using Kryo serialization (set Java property "spark.serializer" to
+ * "spark.KryoSerializer").
+ * Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>
+ *   <master> is the Spark master URL
+ *   <numStream> is the number rawNetworkStreams, which should be same as number
+ *               of work nodes in the cluster
+ *   <host> is "localhost".
+ *   <port> is the port on which RawTextSender is running in the worker nodes.
+ *   <batchMillise> is the Spark Streaming batch duration in milliseconds.
+ */
+
+object RawNetworkGrep {
+  def main(args: Array[String]) {
+    if (args.length != 5) {
+      System.err.println("Usage: RawNetworkGrep <master> <numStreams> <host> <port> <batchMillis>")
+      System.exit(1)
+    }
+
+    val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args
+
+    // Create the context
+    val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    // Warm up the JVMs on master and slave for JIT compilation to kick in
+    RawTextHelper.warmUp(ssc.sparkContext)
+
+    val rawStreams = (1 to numStreams).map(_ =>
+      ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
+    val union = ssc.union(rawStreams)
+    union.filter(_.contains("the")).count().foreach(r =>
+      println("Grep count: " + r.collect().mkString))
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a9642100e3d4886acd337e82e9ca6af049197410
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -0,0 +1,94 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.storage.StorageLevel
+import com.twitter.algebird._
+import spark.streaming.StreamingContext._
+import spark.SparkContext._
+
+/**
+ * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute
+ * windowed and global Top-K estimates of user IDs occurring in a Twitter stream.
+ * <br>
+ *   <strong>Note</strong> that since Algebird's implementation currently only supports Long inputs,
+ *   the example operates on Long IDs. Once the implementation supports other inputs (such as String),
+ *   the same approach could be used for computing popular topics for example.
+ * <p>
+ * <p>
+ *   <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
+ *   This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure
+ *   for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc),
+ *   that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the
+ *   estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold
+ *   percentage of the overall total count.
+ * <p><p>
+ *   Algebird's implementation is a monoid, so we can succinctly merge two CMS instances in the reduce operation.
+ */
+object TwitterAlgebirdCMS {
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println("Usage: TwitterAlgebirdCMS <master> <twitter_username> <twitter_password>" +
+        " [filter1] [filter2] ... [filter n]")
+      System.exit(1)
+    }
+
+    // CMS parameters
+    val DELTA = 1E-3
+    val EPS = 0.01
+    val SEED = 1
+    val PERC = 0.001
+    // K highest frequency elements to take
+    val TOPK = 10
+
+    val Array(master, username, password) = args.slice(0, 3)
+    val filters = args.slice(3, args.length)
+
+    val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+    val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER)
+
+    val users = stream.map(status => status.getUser.getId)
+
+    val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC)
+    var globalCMS = cms.zero
+    val mm = new MapMonoid[Long, Int]()
+    var globalExact = Map[Long, Int]()
+
+    val approxTopUsers = users.mapPartitions(ids => {
+      ids.map(id => cms.create(id))
+    }).reduce(_ ++ _)
+
+    val exactTopUsers = users.map(id => (id, 1))
+      .reduceByKey((a, b) => a + b)
+
+    approxTopUsers.foreach(rdd => {
+      if (rdd.count() != 0) {
+        val partial = rdd.first()
+        val partialTopK = partial.heavyHitters.map(id =>
+          (id, partial.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK)
+        globalCMS ++= partial
+        val globalTopK = globalCMS.heavyHitters.map(id =>
+          (id, globalCMS.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK)
+        println("Approx heavy hitters at %2.2f%% threshold this batch: %s".format(PERC,
+          partialTopK.mkString("[", ",", "]")))
+        println("Approx heavy hitters at %2.2f%% threshold overall: %s".format(PERC,
+          globalTopK.mkString("[", ",", "]")))
+      }
+    })
+
+    exactTopUsers.foreach(rdd => {
+      if (rdd.count() != 0) {
+        val partialMap = rdd.collect().toMap
+        val partialTopK = rdd.map(
+          {case (id, count) => (count, id)})
+          .sortByKey(ascending = false).take(TOPK)
+        globalExact = mm.plus(globalExact.toMap, partialMap)
+        val globalTopK = globalExact.toSeq.sortBy(_._2).reverse.slice(0, TOPK)
+        println("Exact heavy hitters this batch: %s".format(partialTopK.mkString("[", ",", "]")))
+        println("Exact heavy hitters overall: %s".format(globalTopK.mkString("[", ",", "]")))
+      }
+    })
+
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f3288bfb8547de7207c53e7b129f84714bacd718
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -0,0 +1,72 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.storage.StorageLevel
+import com.twitter.algebird.HyperLogLog._
+import com.twitter.algebird.HyperLogLogMonoid
+import spark.streaming.dstream.TwitterInputDStream
+
+/**
+ * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute
+ * a windowed and global estimate of the unique user IDs occurring in a Twitter stream.
+ * <p>
+ * <p>
+ *   This <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
+ *   blog post</a> and this
+ *   <a href="http://highscalability.com/blog/2012/4/5/big-data-counting-how-to-count-a-billion-distinct-objects-us.html">blog post</a>
+ *   have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for estimating
+ *   the cardinality of a data stream, i.e. the number of unique elements.
+ * <p><p>
+ *   Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the reduce operation.
+ */
+object TwitterAlgebirdHLL {
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println("Usage: TwitterAlgebirdHLL <master> <twitter_username> <twitter_password>" +
+        " [filter1] [filter2] ... [filter n]")
+      System.exit(1)
+    }
+
+    /** Bit size parameter for HyperLogLog, trades off accuracy vs size */
+    val BIT_SIZE = 12
+    val Array(master, username, password) = args.slice(0, 3)
+    val filters = args.slice(3, args.length)
+
+    val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+    val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER)
+
+    val users = stream.map(status => status.getUser.getId)
+
+    val hll = new HyperLogLogMonoid(BIT_SIZE)
+    var globalHll = hll.zero
+    var userSet: Set[Long] = Set()
+
+    val approxUsers = users.mapPartitions(ids => {
+      ids.map(id => hll(id))
+    }).reduce(_ + _)
+
+    val exactUsers = users.map(id => Set(id)).reduce(_ ++ _)
+
+    approxUsers.foreach(rdd => {
+      if (rdd.count() != 0) {
+        val partial = rdd.first()
+        globalHll += partial
+        println("Approx distinct users this batch: %d".format(partial.estimatedSize.toInt))
+        println("Approx distinct users overall: %d".format(globalHll.estimatedSize.toInt))
+      }
+    })
+
+    exactUsers.foreach(rdd => {
+      if (rdd.count() != 0) {
+        val partial = rdd.first()
+        userSet ++= partial
+        println("Exact distinct users this batch: %d".format(partial.size))
+        println("Exact distinct users overall: %d".format(userSet.size))
+        println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1) * 100))
+      }
+    })
+
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
new file mode 100644
index 0000000000000000000000000000000000000000..9d4494c6f2759f07d13abe78e93bf7cae5194f0e
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
@@ -0,0 +1,54 @@
+package spark.streaming.examples
+
+import spark.streaming.{Seconds, StreamingContext}
+import StreamingContext._
+import spark.SparkContext._
+
+/**
+ * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter
+ * stream. The stream is instantiated with credentials and optionally filters supplied by the
+ * command line arguments.
+ *
+ */
+object TwitterPopularTags {
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println("Usage: TwitterPopularTags <master> <twitter_username> <twitter_password>" +
+        " [filter1] [filter2] ... [filter n]")
+      System.exit(1)
+    }
+
+    val Array(master, username, password) = args.slice(0, 3)
+    val filters = args.slice(3, args.length)
+
+    val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+    val stream = ssc.twitterStream(username, password, filters)
+
+    val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
+
+    val topCounts60 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(60))
+                     .map{case (topic, count) => (count, topic)}
+                     .transform(_.sortByKey(false))
+
+    val topCounts10 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(10))
+                     .map{case (topic, count) => (count, topic)}
+                     .transform(_.sortByKey(false))
+
+
+    // Print popular hashtags
+    topCounts60.foreach(rdd => {
+      val topList = rdd.take(5)
+      println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count()))
+      topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
+    })
+
+    topCounts10.foreach(rdd => {
+      val topList = rdd.take(5)
+      println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count()))
+      topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))}
+    })
+
+    ssc.start()
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
new file mode 100644
index 0000000000000000000000000000000000000000..74d0d338b754e9ab87807c94a89b5bf6b477587e
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/ZeroMQWordCount.scala
@@ -0,0 +1,74 @@
+package spark.streaming.examples
+
+import akka.actor.ActorSystem
+import akka.actor.actorRef2Scala
+import akka.zeromq._
+import spark.streaming.{ Seconds, StreamingContext }
+import spark.streaming.StreamingContext._
+import akka.zeromq.Subscribe
+
+/**
+ * A simple publisher for demonstration purposes, repeatedly publishes random Messages
+ * every one second.
+ */
+object SimpleZeroMQPublisher {
+
+  def main(args: Array[String]) = {
+    if (args.length < 2) {
+      System.err.println("Usage: SimpleZeroMQPublisher <zeroMQUrl> <topic> ")
+      System.exit(1)
+    }
+
+    val Seq(url, topic) = args.toSeq
+    val acs: ActorSystem = ActorSystem()
+
+    val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url))
+    val messages: Array[String] = Array("words ", "may ", "count ")
+    while (true) {
+      Thread.sleep(1000)
+      pubSocket ! ZMQMessage(Frame(topic) :: messages.map(x => Frame(x.getBytes)).toList)
+    }
+    acs.awaitTermination()
+  }
+}
+
+/**
+ * A sample wordcount with ZeroMQStream stream
+ *
+ * To work with zeroMQ, some native libraries have to be installed.
+ * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide](http://www.zeromq.org/intro:get-the-software)
+ * 
+ * Usage: ZeroMQWordCount <master> <zeroMQurl> <topic>
+ * In local mode, <master> should be 'local[n]' with n > 1
+ *   <zeroMQurl> and <topic> describe where zeroMq publisher is running.
+ *
+ * To run this example locally, you may run publisher as
+ *    `$ ./run spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
+ * and run the example as
+ *    `$ ./run spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
+ */
+object ZeroMQWordCount {
+  def main(args: Array[String]) {
+    if (args.length < 3) {
+      System.err.println(
+        "Usage: ZeroMQWordCount <master> <zeroMQurl> <topic>" +
+          "In local mode, <master> should be 'local[n]' with n > 1")
+      System.exit(1)
+    }
+    val Seq(master, url, topic) = args.toSeq
+
+    // Create the context and set the batch size
+    val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator
+
+    //For this stream, a zeroMQ publisher should be running.
+    val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator)
+    val words = lines.flatMap(_.split(" "))
+    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+    wordCounts.print()
+    ssc.start()
+  }
+
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
new file mode 100644
index 0000000000000000000000000000000000000000..4c6e08bc744099b323f1dc2f2d5d485393c49e72
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -0,0 +1,85 @@
+package spark.streaming.examples.clickstream
+
+import java.net.{InetAddress,ServerSocket,Socket,SocketException}
+import java.io.{InputStreamReader, BufferedReader, PrintWriter}
+import util.Random
+
+/** Represents a page view on a website with associated dimension data.*/
+class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) {
+  override def toString() : String = {
+    "%s\t%s\t%s\t%s\n".format(url, status, zipCode, userID)
+  }
+}
+object PageView {
+  def fromString(in : String) : PageView = {
+    val parts = in.split("\t")
+    new PageView(parts(0), parts(1).toInt, parts(2).toInt, parts(3).toInt)
+  }
+}
+
+/** Generates streaming events to simulate page views on a website.
+  *
+  * This should be used in tandem with PageViewStream.scala. Example:
+  * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * */
+object PageViewGenerator {
+  val pages = Map("http://foo.com/"        -> .7,
+                  "http://foo.com/news"    -> 0.2,
+                  "http://foo.com/contact" -> .1)
+  val httpStatus = Map(200 -> .95,
+                       404 -> .05)
+  val userZipCode = Map(94709 -> .5,
+                        94117 -> .5)
+  val userID = Map((1 to 100).map(_ -> .01):_*)
+
+
+  def pickFromDistribution[T](inputMap : Map[T, Double]) : T = {
+    val rand = new Random().nextDouble()
+    var total = 0.0
+    for ((item, prob) <- inputMap) {
+      total = total + prob
+      if (total > rand) {
+        return item
+      }
+    }
+    return inputMap.take(1).head._1 // Shouldn't get here if probabilities add up to 1.0
+  }
+
+  def getNextClickEvent() : String = {
+    val id = pickFromDistribution(userID)
+    val page = pickFromDistribution(pages)
+    val status = pickFromDistribution(httpStatus)
+    val zipCode = pickFromDistribution(userZipCode)
+    new PageView(page, status, zipCode, id).toString()
+  }
+
+  def main(args : Array[String]) {
+    if (args.length != 2) {
+      System.err.println("Usage: PageViewGenerator <port> <viewsPerSecond>")
+      System.exit(1)
+    }
+    val port = args(0).toInt
+    val viewsPerSecond = args(1).toFloat
+    val sleepDelayMs = (1000.0 / viewsPerSecond).toInt
+    val listener = new ServerSocket(port)
+    println("Listening on port: " + port)
+
+    while (true) {
+      val socket = listener.accept()
+      new Thread() {
+        override def run = {
+          println("Got client connected from: " + socket.getInetAddress)
+          val out = new PrintWriter(socket.getOutputStream(), true)
+
+          while (true) {
+            Thread.sleep(sleepDelayMs)
+            out.write(getNextClickEvent())
+            out.flush()
+          }
+          socket.close()
+        }
+      }.start()
+    }
+  }
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e226a4a73a16ba865229c355ce0826311e549ed3
--- /dev/null
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -0,0 +1,84 @@
+package spark.streaming.examples.clickstream
+
+import spark.streaming.{Seconds, StreamingContext}
+import spark.streaming.StreamingContext._
+import spark.SparkContext._
+
+/** Analyses a streaming dataset of web page views. This class demonstrates several types of
+  * operators available in Spark streaming.
+  *
+  * This should be used in tandem with PageViewStream.scala. Example:
+  * $ ./run spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+  * $ ./run spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+  * */
+object PageViewStream {
+  def main(args: Array[String]) {
+    if (args.length != 3) {
+      System.err.println("Usage: PageViewStream <metric> <host> <port>")
+      System.err.println("<metric> must be one of pageCounts, slidingPageCounts," +
+                         " errorRatePerZipCode, activeUserCount, popularUsersSeen")
+      System.exit(1)
+    }
+    val metric = args(0)
+    val host = args(1)
+    val port = args(2).toInt
+
+    // Create the context
+    val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1),
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+
+    // Create a NetworkInputDStream on target host:port and convert each line to a PageView
+    val pageViews = ssc.socketTextStream(host, port)
+                       .flatMap(_.split("\n"))
+                       .map(PageView.fromString(_))
+
+    // Return a count of views per URL seen in each batch
+    val pageCounts = pageViews.map(view => view.url).countByValue()
+
+    // Return a sliding window of page views per URL in the last ten seconds
+    val slidingPageCounts = pageViews.map(view => view.url)
+                                     .countByValueAndWindow(Seconds(10), Seconds(2))
+
+
+    // Return the rate of error pages (a non 200 status) in each zip code over the last 30 seconds
+    val statusesPerZipCode = pageViews.window(Seconds(30), Seconds(2))
+                                      .map(view => ((view.zipCode, view.status)))
+                                      .groupByKey()
+    val errorRatePerZipCode = statusesPerZipCode.map{
+      case(zip, statuses) =>
+        val normalCount = statuses.filter(_ == 200).size
+        val errorCount = statuses.size - normalCount
+        val errorRatio = errorCount.toFloat / statuses.size
+        if (errorRatio > 0.05) {"%s: **%s**".format(zip, errorRatio)}
+        else {"%s: %s".format(zip, errorRatio)}
+    }
+
+    // Return the number unique users in last 15 seconds
+    val activeUserCount = pageViews.window(Seconds(15), Seconds(2))
+                                   .map(view => (view.userID, 1))
+                                   .groupByKey()
+                                   .count()
+                                   .map("Unique active users: " + _)
+
+    // An external dataset we want to join to this stream
+    val userList = ssc.sparkContext.parallelize(
+       Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq)
+
+    metric match {
+      case "pageCounts" => pageCounts.print()
+      case "slidingPageCounts" => slidingPageCounts.print()
+      case "errorRatePerZipCode" => errorRatePerZipCode.print()
+      case "activeUserCount" => activeUserCount.print()
+      case "popularUsersSeen" =>
+        // Look for users in our existing dataset and print it out if we have a match
+        pageViews.map(view => (view.userID, 1))
+          .foreach((rdd, time) => rdd.join(userList)
+            .map(_._2._2)
+            .take(10)
+            .foreach(u => println("Saw user %s at time %s".format(u, time))))
+      case _ => println("Invalid metric entered: " + metric)
+    }
+
+    ssc.start()
+  }
+}
diff --git a/pom.xml b/pom.xml
index 756fe8783bdbc24e5dd2c7cb42863439a7135dc9..52f92288967e3cae24b65c2c356c9e0ed07acd70 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.0-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>
@@ -41,8 +41,9 @@
     <module>core</module>
     <module>bagel</module>
     <module>examples</module>
-    <!--<module>repl</module>
-    <module>repl-bin</module>-->
+    <module>streaming</module>
+    <module>repl</module>
+    <module>repl-bin</module>
   </modules>
 
   <properties>
@@ -57,6 +58,7 @@
     <spray.json.version>1.2.3</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>
@@ -83,9 +85,9 @@
       </snapshots>
     </repository>
     <repository>
-      <id>cloudera-repo</id>
-      <name>Cloudera Repository</name>
-      <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+      <id>akka-repo</id>
+      <name>Akka Repository</name>
+      <url>http://repo.akka.io/releases/</url>
       <releases>
         <enabled>true</enabled>
       </releases>
@@ -104,6 +106,17 @@
         <enabled>false</enabled>
       </snapshots>
     </repository>
+    <repository>
+      <id>twitter4j-repo</id>
+      <name>Twitter4J Repository</name>
+      <url>http://twitter4j.org/maven2/</url>
+      <releases>
+        <enabled>true</enabled>
+      </releases>
+      <snapshots>
+        <enabled>false</enabled>
+      </snapshots>
+    </repository>
   </repositories>
   <pluginRepositories>
     <pluginRepository>
@@ -186,7 +199,7 @@
       <dependency>
         <groupId>de.javakaffee</groupId>
         <artifactId>kryo-serializers</artifactId>
-        <version>0.20</version>
+        <version>0.22</version>
       </dependency>
       <dependency>
         <groupId>org.scala-lang</groupId>
@@ -260,12 +273,24 @@
         <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.9.1</version>
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <groupId>org.easymock</groupId>
+        <artifactId>easymock</artifactId>
+        <version>3.1</version>
+        <scope>test</scope>
+      </dependency>
       <dependency>
         <groupId>org.scalacheck</groupId>
         <artifactId>scalacheck_${scala.version}</artifactId>
@@ -492,11 +517,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-      <activation>
-        <property>
-          <name>!hadoopVersion</name>
-        </property>
-      </activation>
 
       <properties>
         <hadoop.major.version>1</hadoop.major.version>
@@ -506,7 +526,7 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-core</artifactId>
-            <version>1.0.3</version>
+            <version>1.0.4</version>
           </dependency>
         </dependencies>
       </dependencyManagement>
@@ -514,12 +534,6 @@
 
     <profile>
       <id>hadoop2</id>
-      <activation>
-        <property>
-          <name>hadoopVersion</name>
-          <value>2</value>
-        </property>
-      </activation>
       <properties>
         <hadoop.major.version>2</hadoop.major.version>
       </properties>
@@ -535,6 +549,17 @@
             <artifactId>hadoop-client</artifactId>
             <version>2.0.0-mr1-cdh${cdh.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>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 9fcdbb5b92e19df60af02600124d264abe2b897f..2571e54b048181357add0f669d94ad53c756e0d4 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -9,39 +9,51 @@ import twirl.sbt.TwirlPlugin._
 
 object SparkBuild extends Build {
   // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or
-  // "1.0.3" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop.
-  val HADOOP_VERSION = "1.0.3"
+  // "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"
 
   // 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"
 
-  lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel)
+  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)
+  lazy val repl = Project("repl", file("repl"), settings = replSettings) dependsOn (core) dependsOn (streaming)
 
-  lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core)
+  lazy val examples = Project("examples", file("examples"), settings = examplesSettings) dependsOn (core) dependsOn (streaming)
 
   lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn (core)
 
+  lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core)
+
   // A configuration to set an alternative publishLocalConfiguration
   lazy val MavenCompile = config("m2r") extend(Compile)
   lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
     organization       := "org.spark-project",
-    version            := "0.7.0-SNAPSHOT",
+    version            := "0.8.0-SNAPSHOT",
     scalaVersion       := "2.10.0",
-    scalacOptions      := Seq("-unchecked", "-optimize"),
+    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))),
 
+    // Fork new JVMs for tests and set Java options for those
+    fork := true,
+    javaOptions += "-Xmx1g",
+
+    // 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
     resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
       "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
@@ -90,7 +102,8 @@ object SparkBuild extends Build {
     "org.eclipse.jetty" % "jetty-server"    % "7.5.3.v20111011",
     "org.scalatest"    %% "scalatest"       % "1.9.1"  % "test",
     "org.scalacheck"   %% "scalacheck"      % "1.10.0" % "test",
-    "com.novocode"      % "junit-interface" % "0.8"    % "test"
+    "com.novocode"      % "junit-interface" % "0.8"    % "test",
+    "org.easymock"      % "easymock"        % "3.1"    % "test"
   ),
     parallelExecution := false,
     /* Workaround for issue #206 (fixed after SBT 0.11.0) */
@@ -111,10 +124,11 @@ object SparkBuild extends Build {
   def coreSettings = sharedSettings ++ Seq(
     name := "spark-core",
     resolvers ++= Seq(
-      "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
-      "JBoss Repository"    at "http://repository.jboss.org/nexus/content/repositories/releases/",
-      "Spray Repository"    at "http://repo.spray.cc/",
-      "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
+      "Typesafe Repository"  at "http://repo.typesafe.com/typesafe/releases/",
+      "JBoss Repository"     at "http://repository.jboss.org/nexus/content/repositories/releases/",
+      "Spray Repository"     at "http://repo.spray.cc/",
+      "Cloudera Repository"  at "https://repository.cloudera.com/artifactory/cloudera-repos/",
+      "Twitter4J Repository" at "http://twitter4j.org/maven2/"
     ),
 
     libraryDependencies ++= Seq(
@@ -155,11 +169,22 @@ object SparkBuild extends Build {
   )
 
   def examplesSettings = sharedSettings ++ Seq(
-    name := "spark-examples"
+    name := "spark-examples",
+    libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.11")
   )
 
   def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
 
+  def streamingSettings = sharedSettings ++ Seq(
+    name := "spark-streaming",
+    libraryDependencies ++= Seq(
+      "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile",
+      "com.github.sgroschupf" % "zkclient" % "0.1",
+      "org.twitter4j" % "twitter4j-stream" % "3.0.3",
+      "com.typesafe.akka" % "akka-zeromq" % "2.0.3"
+    )
+  ) ++ assemblySettings ++ extraAssemblySettings
+
   def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq(
     mergeStrategy in assembly := {
       case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
diff --git a/project/build.properties b/project/build.properties
index 4474a03e1aa9671540ef1677b5579ef91481f503..9b860e23c51a6a794509c549acde3ce9ab007233 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -1 +1 @@
-sbt.version=0.12.1
+sbt.version=0.12.3
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 0e4eb7085c01fa6757b185313472d6cb14b268b8..d4f244287236343a0d475849bb3d1ff4c3b6d152 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -4,7 +4,7 @@ resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/release
 
 resolvers += "Spray Repository" at "http://repo.spray.cc/"
 
-addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.3")
+addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.8.5")
 
 addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.1")
 
diff --git a/pyspark b/pyspark
index ab7f4f50c01bd7f34b49a05e8bde24fa756e573d..d662e90287edfb3c45e95cf8f3544ed76ddc82ac 100755
--- a/pyspark
+++ b/pyspark
@@ -36,4 +36,9 @@ if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then
     export SPARK_LAUNCH_WITH_SCALA=1
 fi
 
-exec "$PYSPARK_PYTHON" "$@"
+if [[ "$IPYTHON" = "1" ]] ; then
+    export PYSPARK_PYTHON="ipython"
+    exec "$PYSPARK_PYTHON" -i -c "%run $PYTHONSTARTUP"
+else
+    exec "$PYSPARK_PYTHON" "$@"
+fi
diff --git a/python/epydoc.conf b/python/epydoc.conf
index 91ac984ba294c99844afaca438b6e5fda3ea6e0e..45102cd9fece4bc801bb8577f30b859becf7df07 100644
--- a/python/epydoc.conf
+++ b/python/epydoc.conf
@@ -16,4 +16,4 @@ target: docs/
 private: no
 
 exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.serializers
-         pyspark.java_gateway pyspark.examples pyspark.shell
+         pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test
diff --git a/python/examples/als.py b/python/examples/als.py
new file mode 100755
index 0000000000000000000000000000000000000000..010f80097f1a643f7a69eebac9ad0d048150af22
--- /dev/null
+++ b/python/examples/als.py
@@ -0,0 +1,71 @@
+"""
+This example requires numpy (http://www.numpy.org/)
+"""
+from os.path import realpath
+import sys
+
+import numpy as np
+from numpy.random import rand
+from numpy import matrix
+from pyspark import SparkContext
+
+LAMBDA = 0.01   # regularization
+np.random.seed(42)
+
+def rmse(R, ms, us):
+    diff = R - ms * us.T
+    return np.sqrt(np.sum(np.power(diff, 2)) / M * U)
+
+def update(i, vec, mat, ratings):
+    uu = mat.shape[0]
+    ff = mat.shape[1]
+    XtX = matrix(np.zeros((ff, ff)))
+    Xty = np.zeros((ff, 1))
+
+    for j in range(uu):
+        v = mat[j, :]
+        XtX += v.T * v
+        Xty += v.T * ratings[i, j]
+    XtX += np.eye(ff, ff) * LAMBDA * uu
+    return np.linalg.solve(XtX, Xty)
+
+if __name__ == "__main__":
+    if len(sys.argv) < 2:
+        print >> sys.stderr, \
+            "Usage: PythonALS <master> <M> <U> <F> <iters> <slices>"
+        exit(-1)
+    sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)])
+    M = int(sys.argv[2]) if len(sys.argv) > 2 else 100
+    U = int(sys.argv[3]) if len(sys.argv) > 3 else 500
+    F = int(sys.argv[4]) if len(sys.argv) > 4 else 10
+    ITERATIONS = int(sys.argv[5]) if len(sys.argv) > 5 else 5
+    slices = int(sys.argv[6]) if len(sys.argv) > 6 else 2
+
+    print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \
+            (M, U, F, ITERATIONS, slices)
+
+    R = matrix(rand(M, F)) * matrix(rand(U, F).T)
+    ms = matrix(rand(M ,F))
+    us = matrix(rand(U, F))
+
+    Rb = sc.broadcast(R)
+    msb = sc.broadcast(ms)
+    usb = sc.broadcast(us)
+
+    for i in range(ITERATIONS):
+        ms = sc.parallelize(range(M), slices) \
+               .map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \
+               .collect()
+        ms = matrix(np.array(ms)[:, :, 0])      # collect() returns a list, so array ends up being
+                                                # a 3-d array, we take the first 2 dims for the matrix
+        msb = sc.broadcast(ms)
+
+        us = sc.parallelize(range(U), slices) \
+               .map(lambda x: update(x, usb.value[x, :], msb.value, Rb.value.T)) \
+               .collect()
+        us = matrix(np.array(us)[:, :, 0])
+        usb = sc.broadcast(us)
+
+        error = rmse(R, ms, us)
+        print "Iteration %d:" % i 
+        print "\nRMSE: %5.4f\n" % error
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/__init__.py b/python/pyspark/__init__.py
index c595ae084255a26de62b60912eecc20bdde20455..3e8bca62f069f48607ac470e1a79eb3710b5700c 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -7,6 +7,12 @@ Public classes:
         Main entry point for Spark functionality.
     - L{RDD<pyspark.rdd.RDD>}
         A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
+    - L{Broadcast<pyspark.broadcast.Broadcast>}
+        A broadcast variable that gets reused across tasks.
+    - L{Accumulator<pyspark.accumulators.Accumulator>}
+        An "add-only" shared variable that tasks can only add values to.
+    - L{SparkFiles<pyspark.files.SparkFiles>}
+        Access files shipped with jobs.
 """
 import sys
 import os
@@ -15,6 +21,7 @@ sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.eg
 
 from pyspark.context import SparkContext
 from pyspark.rdd import RDD
+from pyspark.files import SparkFiles
 
 
-__all__ = ["SparkContext", "RDD"]
+__all__ = ["SparkContext", "RDD", "SparkFiles"]
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
new file mode 100644
index 0000000000000000000000000000000000000000..3e9d7d36da8a1b0fb81661a8ee0d02d076072939
--- /dev/null
+++ b/python/pyspark/accumulators.py
@@ -0,0 +1,198 @@
+"""
+>>> from pyspark.context import SparkContext
+>>> sc = SparkContext('local', 'test')
+>>> a = sc.accumulator(1)
+>>> a.value
+1
+>>> a.value = 2
+>>> a.value
+2
+>>> a += 5
+>>> a.value
+7
+
+>>> sc.accumulator(1.0).value
+1.0
+
+>>> sc.accumulator(1j).value
+1j
+
+>>> rdd = sc.parallelize([1,2,3])
+>>> def f(x):
+...     global a
+...     a += x
+>>> rdd.foreach(f)
+>>> a.value
+13
+
+>>> from pyspark.accumulators import AccumulatorParam
+>>> class VectorAccumulatorParam(AccumulatorParam):
+...     def zero(self, value):
+...         return [0.0] * len(value)
+...     def addInPlace(self, val1, val2):
+...         for i in xrange(len(val1)):
+...              val1[i] += val2[i]
+...         return val1
+>>> va = sc.accumulator([1.0, 2.0, 3.0], VectorAccumulatorParam())
+>>> va.value
+[1.0, 2.0, 3.0]
+>>> def g(x):
+...     global va
+...     va += [x] * 3
+>>> rdd.foreach(g)
+>>> va.value
+[7.0, 8.0, 9.0]
+
+>>> rdd.map(lambda x: a.value).collect() # doctest: +IGNORE_EXCEPTION_DETAIL
+Traceback (most recent call last):
+    ...
+Py4JJavaError:...
+
+>>> def h(x):
+...     global a
+...     a.value = 7
+>>> rdd.foreach(h) # doctest: +IGNORE_EXCEPTION_DETAIL
+Traceback (most recent call last):
+    ...
+Py4JJavaError:...
+
+>>> sc.accumulator([1.0, 2.0, 3.0]) # doctest: +IGNORE_EXCEPTION_DETAIL
+Traceback (most recent call last):
+    ...
+Exception:...
+"""
+
+import struct
+import SocketServer
+import threading
+from pyspark.cloudpickle import CloudPickler
+from pyspark.serializers import read_int, read_with_length, load_pickle
+
+
+# Holds accumulators registered on the current machine, keyed by ID. This is then used to send
+# the local accumulator updates back to the driver program at the end of a task.
+_accumulatorRegistry = {}
+
+
+def _deserialize_accumulator(aid, zero_value, accum_param):
+    from pyspark.accumulators import _accumulatorRegistry
+    accum = Accumulator(aid, zero_value, accum_param)
+    accum._deserialized = True
+    _accumulatorRegistry[aid] = accum
+    return accum
+
+
+class Accumulator(object):
+    """
+    A shared variable that can be accumulated, i.e., has a commutative and associative "add"
+    operation. Worker tasks on a Spark cluster can add values to an Accumulator with the C{+=}
+    operator, but only the driver program is allowed to access its value, using C{value}.
+    Updates from the workers get propagated automatically to the driver program.
+
+    While C{SparkContext} supports accumulators for primitive data types like C{int} and
+    C{float}, users can also define accumulators for custom types by providing a custom
+    L{AccumulatorParam} object. Refer to the doctest of this module for an example.
+    """
+
+    def __init__(self, aid, value, accum_param):
+        """Create a new Accumulator with a given initial value and AccumulatorParam object"""
+        from pyspark.accumulators import _accumulatorRegistry
+        self.aid = aid
+        self.accum_param = accum_param
+        self._value = value
+        self._deserialized = False
+        _accumulatorRegistry[aid] = self
+
+    def __reduce__(self):
+        """Custom serialization; saves the zero value from our AccumulatorParam"""
+        param = self.accum_param
+        return (_deserialize_accumulator, (self.aid, param.zero(self._value), param))
+
+    @property
+    def value(self):
+        """Get the accumulator's value; only usable in driver program"""
+        if self._deserialized:
+            raise Exception("Accumulator.value cannot be accessed inside tasks")
+        return self._value
+
+    @value.setter
+    def value(self, value):
+        """Sets the accumulator's value; only usable in driver program"""
+        if self._deserialized:
+            raise Exception("Accumulator.value cannot be accessed inside tasks")
+        self._value = value
+
+    def __iadd__(self, term):
+        """The += operator; adds a term to this accumulator's value"""
+        self._value = self.accum_param.addInPlace(self._value, term)
+        return self
+
+    def __str__(self):
+        return str(self._value)
+
+    def __repr__(self):
+        return "Accumulator<id=%i, value=%s>" % (self.aid, self._value)
+
+
+class AccumulatorParam(object):
+    """
+    Helper object that defines how to accumulate values of a given type.
+    """
+
+    def zero(self, value):
+        """
+        Provide a "zero value" for the type, compatible in dimensions with the
+        provided C{value} (e.g., a zero vector)
+        """
+        raise NotImplementedError
+
+    def addInPlace(self, value1, value2):
+        """
+        Add two values of the accumulator's data type, returning a new value;
+        for efficiency, can also update C{value1} in place and return it.
+        """
+        raise NotImplementedError
+
+
+class AddingAccumulatorParam(AccumulatorParam):
+    """
+    An AccumulatorParam that uses the + operators to add values. Designed for simple types
+    such as integers, floats, and lists. Requires the zero value for the underlying type
+    as a parameter.
+    """
+
+    def __init__(self, zero_value):
+        self.zero_value = zero_value
+
+    def zero(self, value):
+        return self.zero_value
+
+    def addInPlace(self, value1, value2):
+        value1 += value2
+        return value1
+
+
+# Singleton accumulator params for some standard types
+INT_ACCUMULATOR_PARAM = AddingAccumulatorParam(0)
+FLOAT_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0)
+COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j)
+
+
+class _UpdateRequestHandler(SocketServer.StreamRequestHandler):
+    def handle(self):
+        from pyspark.accumulators import _accumulatorRegistry
+        num_updates = read_int(self.rfile)
+        for _ in range(num_updates):
+            (aid, update) = load_pickle(read_with_length(self.rfile))
+            _accumulatorRegistry[aid] += update
+        # Write a byte in acknowledgement
+        self.wfile.write(struct.pack("!b", 1))
+
+
+def _start_update_server():
+    """Start a TCP server to receive accumulator updates in a daemon thread, and returns it"""
+    server = SocketServer.TCPServer(("localhost", 0), _UpdateRequestHandler)
+    thread = threading.Thread(target=server.serve_forever)
+    thread.daemon = True
+    thread.start()
+    return server
diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py
index 93876fa738f6336f0e9a21c484acf7dd74ce0477..def810dd461dab770ef5b5b351beaf6446b381e7 100644
--- a/python/pyspark/broadcast.py
+++ b/python/pyspark/broadcast.py
@@ -37,12 +37,3 @@ class Broadcast(object):
     def __reduce__(self):
         self._pickle_registry.add(self)
         return (_from_id, (self.bid, ))
-
-
-def _test():
-    import doctest
-    doctest.testmod()
-
-
-if __name__ == "__main__":
-    _test()
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index e486f206b09cfeca6593551e4f2d747ad3db9ec1..657fe6f98975bfa2ea86bc6435a868f0a97d4661 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -1,8 +1,13 @@
 import os
-import atexit
+import shutil
+import sys
+from threading import Lock
 from tempfile import NamedTemporaryFile
 
+from pyspark import accumulators
+from pyspark.accumulators import Accumulator
 from pyspark.broadcast import Broadcast
+from pyspark.files import SparkFiles
 from pyspark.java_gateway import launch_gateway
 from pyspark.serializers import dump_pickle, write_with_length, batched
 from pyspark.rdd import RDD
@@ -17,11 +22,13 @@ class SparkContext(object):
     broadcast variables on that cluster.
     """
 
-    gateway = launch_gateway()
-    jvm = gateway.jvm
-    _readRDDFromPickleFile = jvm.PythonRDD.readRDDFromPickleFile
-    _writeIteratorToPickleFile = jvm.PythonRDD.writeIteratorToPickleFile
-    _takePartition = jvm.PythonRDD.takePartition
+    _gateway = None
+    _jvm = None
+    _writeIteratorToPickleFile = None
+    _takePartition = None
+    _next_accum_id = 0
+    _active_spark_context = None
+    _lock = Lock()
 
     def __init__(self, master, jobName, sparkHome=None, pyFiles=None,
         environment=None, batchSize=1024):
@@ -41,6 +48,18 @@ class SparkContext(object):
                Java object.  Set 1 to disable batching or -1 to use an
                unlimited batch size.
         """
+        with SparkContext._lock:
+            if SparkContext._active_spark_context:
+                raise ValueError("Cannot run multiple SparkContexts at once")
+            else:
+                SparkContext._active_spark_context = self
+                if not SparkContext._gateway:
+                    SparkContext._gateway = launch_gateway()
+                    SparkContext._jvm = SparkContext._gateway.jvm
+                    SparkContext._writeIteratorToPickleFile = \
+                        SparkContext._jvm.PythonRDD.writeIteratorToPickleFile
+                    SparkContext._takePartition = \
+                        SparkContext._jvm.PythonRDD.takePartition
         self.master = master
         self.jobName = jobName
         self.sparkHome = sparkHome or None # None becomes null in Py4J
@@ -48,10 +67,18 @@ class SparkContext(object):
         self.batchSize = batchSize  # -1 represents a unlimited batch size
 
         # Create the Java SparkContext through Py4J
-        empty_string_array = self.gateway.new_array(self.jvm.String, 0)
-        self._jsc = self.jvm.JavaSparkContext(master, jobName, sparkHome,
+        empty_string_array = self._gateway.new_array(self._jvm.String, 0)
+        self._jsc = self._jvm.JavaSparkContext(master, jobName, sparkHome,
                                               empty_string_array)
 
+        # Create a single Accumulator in Java that we'll send all our updates through;
+        # they will be passed back to us through a TCP server
+        self._accumulatorServer = accumulators._start_update_server()
+        (host, port) = self._accumulatorServer.server_address
+        self._javaAccumulator = self._jsc.accumulator(
+                self._jvm.java.util.ArrayList(),
+                self._jvm.PythonAccumulatorParam(host, port))
+
         self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
         # Broadcast's __reduce__ method stores Broadcast instances here.
         # This allows other code to determine which Broadcast instances have
@@ -62,6 +89,13 @@ class SparkContext(object):
         # Deploy any code dependencies specified in the constructor
         for path in (pyFiles or []):
             self.addPyFile(path)
+        SparkFiles._sc = self
+        sys.path.append(SparkFiles.getRootDirectory())
+
+        # Create a temporary directory inside spark.local.dir:
+        local_dir = self._jvm.spark.Utils.getLocalDir()
+        self._temp_dir = \
+            self._jvm.spark.Utils.createTempDir(local_dir).getAbsolutePath()
 
     @property
     def defaultParallelism(self):
@@ -72,15 +106,20 @@ class SparkContext(object):
         return self._jsc.sc().defaultParallelism()
 
     def __del__(self):
-        if self._jsc:
-            self._jsc.stop()
+        self.stop()
 
     def stop(self):
         """
         Shut down the SparkContext.
         """
-        self._jsc.stop()
-        self._jsc = None
+        if self._jsc:
+            self._jsc.stop()
+            self._jsc = None
+        if self._accumulatorServer:
+            self._accumulatorServer.shutdown()
+            self._accumulatorServer = None
+        with SparkContext._lock:
+            SparkContext._active_spark_context = None
 
     def parallelize(self, c, numSlices=None):
         """
@@ -90,14 +129,14 @@ class SparkContext(object):
         # Calling the Java parallelize() method with an ArrayList is too slow,
         # because it sends O(n) Py4J commands.  As an alternative, serialized
         # objects are written to a file and loaded through textFile().
-        tempFile = NamedTemporaryFile(delete=False)
-        atexit.register(lambda: os.unlink(tempFile.name))
+        tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir)
         if self.batchSize != 1:
             c = batched(c, self.batchSize)
         for x in c:
             write_with_length(dump_pickle(x), tempFile)
         tempFile.close()
-        jrdd = self._readRDDFromPickleFile(self._jsc, tempFile.name, numSlices)
+        readRDDFromPickleFile = self._jvm.PythonRDD.readRDDFromPickleFile
+        jrdd = readRDDFromPickleFile(self._jsc, tempFile.name, numSlices)
         return RDD(jrdd, self)
 
     def textFile(self, name, minSplits=None):
@@ -110,6 +149,10 @@ class SparkContext(object):
         jrdd = self._jsc.textFile(name, minSplits)
         return RDD(jrdd, self)
 
+    def _checkpointFile(self, name):
+        jrdd = self._jsc.checkpointFile(name)
+        return RDD(jrdd, self)
+
     def union(self, rdds):
         """
         Build the union of a list of RDDs.
@@ -129,12 +172,48 @@ class SparkContext(object):
         return Broadcast(jbroadcast.id(), value, jbroadcast,
                          self._pickled_broadcast_vars)
 
+    def accumulator(self, value, accum_param=None):
+        """
+        Create an L{Accumulator} with the given initial value, using a given
+        L{AccumulatorParam} helper object to define how to add values of the
+        data type if provided. Default AccumulatorParams are used for integers
+        and floating-point numbers if you do not provide one. For other types,
+        a custom AccumulatorParam can be used.
+        """
+        if accum_param == None:
+            if isinstance(value, int):
+                accum_param = accumulators.INT_ACCUMULATOR_PARAM
+            elif isinstance(value, float):
+                accum_param = accumulators.FLOAT_ACCUMULATOR_PARAM
+            elif isinstance(value, complex):
+                accum_param = accumulators.COMPLEX_ACCUMULATOR_PARAM
+            else:
+                raise Exception("No default accumulator param for type %s" % type(value))
+        SparkContext._next_accum_id += 1
+        return Accumulator(SparkContext._next_accum_id - 1, value, accum_param)
+
     def addFile(self, path):
         """
-        Add a file to be downloaded into the working directory of this Spark
-        job on every node. The C{path} passed can be either a local file,
-        a file in HDFS (or other Hadoop-supported filesystems), or an HTTP,
-        HTTPS or FTP URI.
+        Add a file to be downloaded with this Spark job on every node.
+        The C{path} passed can be either a local file, a file in HDFS
+        (or other Hadoop-supported filesystems), or an HTTP, HTTPS or
+        FTP URI.
+
+        To access the file in Spark jobs, use
+        L{SparkFiles.get(path)<pyspark.files.SparkFiles.get>} to find its
+        download location.
+
+        >>> from pyspark import SparkFiles
+        >>> path = os.path.join(tempdir, "test.txt")
+        >>> with open(path, "w") as testFile:
+        ...    testFile.write("100")
+        >>> sc.addFile(path)
+        >>> def func(iterator):
+        ...    with open(SparkFiles.get("test.txt")) as testFile:
+        ...        fileVal = int(testFile.readline())
+        ...        return [x * 100 for x in iterator]
+        >>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect()
+        [100, 200, 300, 400]
         """
         self._jsc.sc().addFile(path)
 
@@ -155,5 +234,33 @@ class SparkContext(object):
         """
         self.addFile(path)
         filename = path.split("/")[-1]
-        os.environ["PYTHONPATH"] = \
-            "%s:%s" % (filename, os.environ["PYTHONPATH"])
+
+    def setCheckpointDir(self, dirName, useExisting=False):
+        """
+        Set the directory under which RDDs are going to be checkpointed. The
+        directory must be a HDFS path if running on a cluster.
+
+        If the directory does not exist, it will be created. If the directory
+        exists and C{useExisting} is set to true, then the exisiting directory
+        will be used.  Otherwise an exception will be thrown to prevent
+        accidental overriding of checkpoint files in the existing directory.
+        """
+        self._jsc.sc().setCheckpointDir(dirName, useExisting)
+
+
+def _test():
+    import atexit
+    import doctest
+    import tempfile
+    globs = globals().copy()
+    globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+    globs['tempdir'] = tempfile.mkdtemp()
+    atexit.register(lambda: shutil.rmtree(globs['tempdir']))
+    (failure_count, test_count) = doctest.testmod(globs=globs)
+    globs['sc'].stop()
+    if failure_count:
+        exit(-1)
+
+
+if __name__ == "__main__":
+    _test()
diff --git a/python/pyspark/files.py b/python/pyspark/files.py
new file mode 100644
index 0000000000000000000000000000000000000000..001b7a28b6691b164c8028bf6f83b49450a8fa13
--- /dev/null
+++ b/python/pyspark/files.py
@@ -0,0 +1,38 @@
+import os
+
+
+class SparkFiles(object):
+    """
+    Resolves paths to files added through
+    L{SparkContext.addFile()<pyspark.context.SparkContext.addFile>}.
+
+    SparkFiles contains only classmethods; users should not create SparkFiles
+    instances.
+    """
+
+    _root_directory = None
+    _is_running_on_worker = False
+    _sc = None
+
+    def __init__(self):
+        raise NotImplementedError("Do not construct SparkFiles objects")
+
+    @classmethod
+    def get(cls, filename):
+        """
+        Get the absolute path of a file added through C{SparkContext.addFile()}.
+        """
+        path = os.path.join(SparkFiles.getRootDirectory(), filename)
+        return os.path.abspath(path)
+
+    @classmethod
+    def getRootDirectory(cls):
+        """
+        Get the root directory that contains files added through
+        C{SparkContext.addFile()}.
+        """
+        if cls._is_running_on_worker:
+            return cls._root_directory
+        else:
+            # This will have to change if we support multiple SparkContexts:
+            return cls._sc._jvm.spark.SparkFiles.getRootDirectory()
diff --git a/python/pyspark/join.py b/python/pyspark/join.py
index 7036c479806679d539a4779df92ed1bfd94bf694..5f4294fb1b7778090584041286822eabbb0e1efe 100644
--- a/python/pyspark/join.py
+++ b/python/pyspark/join.py
@@ -32,13 +32,13 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 """
 
 
-def _do_python_join(rdd, other, numSplits, dispatch):
+def _do_python_join(rdd, other, numPartitions, dispatch):
     vs = rdd.map(lambda (k, v): (k, (1, v)))
     ws = other.map(lambda (k, v): (k, (2, v)))
-    return vs.union(ws).groupByKey(numSplits).flatMapValues(dispatch)
+    return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch)
 
 
-def python_join(rdd, other, numSplits):
+def python_join(rdd, other, numPartitions):
     def dispatch(seq):
         vbuf, wbuf = [], []
         for (n, v) in seq:
@@ -47,10 +47,10 @@ def python_join(rdd, other, numSplits):
             elif n == 2:
                 wbuf.append(v)
         return [(v, w) for v in vbuf for w in wbuf]
-    return _do_python_join(rdd, other, numSplits, dispatch)
+    return _do_python_join(rdd, other, numPartitions, dispatch)
 
 
-def python_right_outer_join(rdd, other, numSplits):
+def python_right_outer_join(rdd, other, numPartitions):
     def dispatch(seq):
         vbuf, wbuf = [], []
         for (n, v) in seq:
@@ -61,10 +61,10 @@ def python_right_outer_join(rdd, other, numSplits):
         if not vbuf:
             vbuf.append(None)
         return [(v, w) for v in vbuf for w in wbuf]
-    return _do_python_join(rdd, other, numSplits, dispatch)
+    return _do_python_join(rdd, other, numPartitions, dispatch)
 
 
-def python_left_outer_join(rdd, other, numSplits):
+def python_left_outer_join(rdd, other, numPartitions):
     def dispatch(seq):
         vbuf, wbuf = [], []
         for (n, v) in seq:
@@ -75,10 +75,10 @@ def python_left_outer_join(rdd, other, numSplits):
         if not wbuf:
             wbuf.append(None)
         return [(v, w) for v in vbuf for w in wbuf]
-    return _do_python_join(rdd, other, numSplits, dispatch)
+    return _do_python_join(rdd, other, numPartitions, dispatch)
 
 
-def python_cogroup(rdd, other, numSplits):
+def python_cogroup(rdd, other, numPartitions):
     vs = rdd.map(lambda (k, v): (k, (1, v)))
     ws = other.map(lambda (k, v): (k, (2, v)))
     def dispatch(seq):
@@ -89,4 +89,4 @@ def python_cogroup(rdd, other, numSplits):
             elif n == 2:
                 wbuf.append(v)
         return (vbuf, wbuf)
-    return vs.union(ws).groupByKey(numSplits).mapValues(dispatch)
+    return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch)
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 1d36da42b0282d61d7cde132ee280c2dcbc5beb3..a9fec17a9da69fb4ce1f7153d4ee31e60a6cb2f8 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -1,4 +1,3 @@
-import atexit
 from base64 import standard_b64encode as b64enc
 import copy
 from collections import defaultdict
@@ -32,7 +31,9 @@ class RDD(object):
     def __init__(self, jrdd, ctx):
         self._jrdd = jrdd
         self.is_cached = False
+        self.is_checkpointed = False
         self.ctx = ctx
+        self._partitionFunc = None
 
     @property
     def context(self):
@@ -49,6 +50,34 @@ class RDD(object):
         self._jrdd.cache()
         return self
 
+    def checkpoint(self):
+        """
+        Mark this RDD for checkpointing. It will be saved to a file inside the
+        checkpoint directory set with L{SparkContext.setCheckpointDir()} and
+        all references to its parent RDDs will be removed. This function must
+        be called before any job has been executed on this RDD. It is strongly
+        recommended that this RDD is persisted in memory, otherwise saving it
+        on a file will require recomputation.
+        """
+        self.is_checkpointed = True
+        self._jrdd.rdd().checkpoint()
+
+    def isCheckpointed(self):
+        """
+        Return whether this RDD has been checkpointed or not
+        """
+        return self._jrdd.rdd().isCheckpointed()
+
+    def getCheckpointFile(self):
+        """
+        Gets the name of the file to which this RDD was checkpointed
+        """
+        checkpointFile = self._jrdd.rdd().getCheckpointFile()
+        if checkpointFile.isDefined():
+            return checkpointFile.get()
+        else:
+            return None
+
     # TODO persist(self, storageLevel)
 
     def map(self, f, preservesPartitioning=False):
@@ -186,7 +215,7 @@ class RDD(object):
                 yield pair
         return java_cartesian.flatMap(unpack_batches)
 
-    def groupBy(self, f, numSplits=None):
+    def groupBy(self, f, numPartitions=None):
         """
         Return an RDD of grouped items.
 
@@ -195,7 +224,7 @@ class RDD(object):
         >>> sorted([(x, sorted(y)) for (x, y) in result])
         [(0, [2, 8]), (1, [1, 1, 3, 5])]
         """
-        return self.map(lambda x: (f(x), x)).groupByKey(numSplits)
+        return self.map(lambda x: (f(x), x)).groupByKey(numPartitions)
 
     def pipe(self, command, env={}):
         """
@@ -234,12 +263,8 @@ class RDD(object):
         # Transferring lots of data through Py4J can be slow because
         # socket.readline() is inefficient.  Instead, we'll dump the data to a
         # file and read it back.
-        tempFile = NamedTemporaryFile(delete=False)
+        tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir)
         tempFile.close()
-        def clean_up_file():
-            try: os.unlink(tempFile.name)
-            except: pass
-        atexit.register(clean_up_file)
         self.ctx._writeIteratorToPickleFile(iterator, tempFile.name)
         # Read the data into Python and deserialize it:
         with open(tempFile.name, 'rb') as tempFile:
@@ -249,8 +274,8 @@ class RDD(object):
 
     def reduce(self, f):
         """
-        Reduces the elements of this RDD using the specified associative binary
-        operator.
+        Reduces the elements of this RDD using the specified commutative and
+        associative binary operator.
 
         >>> from operator import add
         >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add)
@@ -347,6 +372,10 @@ class RDD(object):
         items = []
         for partition in range(self._jrdd.splits().size()):
             iterator = self.ctx._takePartition(self._jrdd.rdd(), partition)
+            # Each item in the iterator is a string, Python object, batch of
+            # Python objects.  Regardless, it is sufficient to take `num`
+            # of these objects in order to collect `num` Python objects:
+            iterator = iterator.take(num)
             items.extend(self._collect_iterator_through_file(iterator))
             if len(items) >= num:
                 break
@@ -370,14 +399,14 @@ 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):
             return (str(x).encode("utf-8") for x in iterator)
         keyed = PipelinedRDD(self, func)
         keyed._bypass_serializer = True
-        keyed._jrdd.map(self.ctx.jvm.BytesToString()).saveAsTextFile(path)
+        keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path)
 
     # Pair functions
 
@@ -393,22 +422,22 @@ class RDD(object):
         """
         return dict(self.collect())
 
-    def reduceByKey(self, func, numSplits=None):
+    def reduceByKey(self, func, numPartitions=None):
         """
         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 "combiner" in MapReduce.
 
-        Output will be hash-partitioned with C{numSplits} splits, or the
-        default parallelism level if C{numSplits} is not specified.
+        Output will be hash-partitioned with C{numPartitions} partitions, or
+        the default parallelism level if C{numPartitions} is not specified.
 
         >>> from operator import add
         >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
         >>> sorted(rdd.reduceByKey(add).collect())
         [('a', 2), ('b', 1)]
         """
-        return self.combineByKey(lambda x: x, func, func, numSplits)
+        return self.combineByKey(lambda x: x, func, func, numPartitions)
 
     def reduceByKeyLocally(self, func):
         """
@@ -445,7 +474,7 @@ class RDD(object):
         """
         return self.map(lambda x: x[0]).countByValue()
 
-    def join(self, other, numSplits=None):
+    def join(self, other, numPartitions=None):
         """
         Return an RDD containing all pairs of elements with matching keys in
         C{self} and C{other}.
@@ -460,9 +489,9 @@ class RDD(object):
         >>> sorted(x.join(y).collect())
         [('a', (1, 2)), ('a', (1, 3))]
         """
-        return python_join(self, other, numSplits)
+        return python_join(self, other, numPartitions)
 
-    def leftOuterJoin(self, other, numSplits=None):
+    def leftOuterJoin(self, other, numPartitions=None):
         """
         Perform a left outer join of C{self} and C{other}.
 
@@ -477,9 +506,9 @@ class RDD(object):
         >>> sorted(x.leftOuterJoin(y).collect())
         [('a', (1, 2)), ('b', (4, None))]
         """
-        return python_left_outer_join(self, other, numSplits)
+        return python_left_outer_join(self, other, numPartitions)
 
-    def rightOuterJoin(self, other, numSplits=None):
+    def rightOuterJoin(self, other, numPartitions=None):
         """
         Perform a right outer join of C{self} and C{other}.
 
@@ -494,10 +523,10 @@ class RDD(object):
         >>> sorted(y.rightOuterJoin(x).collect())
         [('a', (2, 1)), ('b', (None, 4))]
         """
-        return python_right_outer_join(self, other, numSplits)
+        return python_right_outer_join(self, other, numPartitions)
 
     # TODO: add option to control map-side combining
-    def partitionBy(self, numSplits, hashFunc=hash):
+    def partitionBy(self, numPartitions, partitionFunc=hash):
         """
         Return a copy of the RDD partitioned using the specified partitioner.
 
@@ -506,29 +535,33 @@ class RDD(object):
         >>> set(sets[0]).intersection(set(sets[1]))
         set([])
         """
-        if numSplits is None:
-            numSplits = self.ctx.defaultParallelism
+        if numPartitions is None:
+            numPartitions = self.ctx.defaultParallelism
         # Transferring O(n) objects to Java is too expensive.  Instead, we'll
-        # form the hash buckets in Python, transferring O(numSplits) objects
+        # form the hash buckets in Python, transferring O(numPartitions) objects
         # to Java.  Each object is a (splitNumber, [objects]) pair.
         def add_shuffle_key(split, iterator):
             buckets = defaultdict(list)
             for (k, v) in iterator:
-                buckets[hashFunc(k) % numSplits].append((k, v))
+                buckets[partitionFunc(k) % numPartitions].append((k, v))
             for (split, items) in buckets.iteritems():
                 yield str(split)
                 yield dump_pickle(Batch(items))
         keyed = PipelinedRDD(self, add_shuffle_key)
         keyed._bypass_serializer = True
-        pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
-        partitioner = self.ctx.jvm.spark.api.python.PythonPartitioner(numSplits)
-        jrdd = pairRDD.partitionBy(partitioner)
-        jrdd = jrdd.map(self.ctx.jvm.ExtractValue())
-        return RDD(jrdd, self.ctx)
+        pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
+        partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
+                                                     id(partitionFunc))
+        jrdd = pairRDD.partitionBy(partitioner).values()
+        rdd = RDD(jrdd, self.ctx)
+        # This is required so that id(partitionFunc) remains unique, even if
+        # partitionFunc is a lambda:
+        rdd._partitionFunc = partitionFunc
+        return rdd
 
     # TODO: add control over map-side aggregation
     def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
-                     numSplits=None):
+                     numPartitions=None):
         """
         Generic function to combine the elements for each key using a custom
         set of aggregation functions.
@@ -553,8 +586,8 @@ class RDD(object):
         >>> sorted(x.combineByKey(str, add, add).collect())
         [('a', '11'), ('b', '1')]
         """
-        if numSplits is None:
-            numSplits = self.ctx.defaultParallelism
+        if numPartitions is None:
+            numPartitions = self.ctx.defaultParallelism
         def combineLocally(iterator):
             combiners = {}
             for (k, v) in iterator:
@@ -564,7 +597,7 @@ class RDD(object):
                     combiners[k] = mergeValue(combiners[k], v)
             return combiners.iteritems()
         locally_combined = self.mapPartitions(combineLocally)
-        shuffled = locally_combined.partitionBy(numSplits)
+        shuffled = locally_combined.partitionBy(numPartitions)
         def _mergeCombiners(iterator):
             combiners = {}
             for (k, v) in iterator:
@@ -576,10 +609,10 @@ class RDD(object):
         return shuffled.mapPartitions(_mergeCombiners)
 
     # TODO: support variant with custom partitioner
-    def groupByKey(self, numSplits=None):
+    def groupByKey(self, numPartitions=None):
         """
         Group the values for each key in the RDD into a single sequence.
-        Hash-partitions the resulting RDD with into numSplits partitions.
+        Hash-partitions the resulting RDD with into numPartitions partitions.
 
         >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)])
         >>> sorted(x.groupByKey().collect())
@@ -597,7 +630,7 @@ class RDD(object):
             return a + b
 
         return self.combineByKey(createCombiner, mergeValue, mergeCombiners,
-                numSplits)
+                numPartitions)
 
     # TODO: add tests
     def flatMapValues(self, f):
@@ -626,7 +659,7 @@ class RDD(object):
         return self.cogroup(other)
 
     # TODO: add variant with custom parittioner
-    def cogroup(self, other, numSplits=None):
+    def cogroup(self, other, numPartitions=None):
         """
         For each key k in C{self} or C{other}, return a resulting RDD that
         contains a tuple with the list of values for that key in C{self} as well
@@ -637,7 +670,7 @@ class RDD(object):
         >>> sorted(x.cogroup(y).collect())
         [('a', ([1], [2])), ('b', ([4], []))]
         """
-        return python_cogroup(self, other, numSplits)
+        return python_cogroup(self, other, numPartitions)
 
     # TODO: `lookup` is disabled because we can't make direct comparisons based
     # on the key; we need to compare the hash of the key to the hash of the
@@ -662,7 +695,7 @@ class PipelinedRDD(RDD):
     20
     """
     def __init__(self, prev, func, preservesPartitioning=False):
-        if isinstance(prev, PipelinedRDD) and not prev.is_cached:
+        if isinstance(prev, PipelinedRDD) and prev._is_pipelinable():
             prev_func = prev.func
             def pipeline_func(split, iterator):
                 return func(split, prev_func(split, iterator))
@@ -675,6 +708,7 @@ class PipelinedRDD(RDD):
             self.preservesPartitioning = preservesPartitioning
             self._prev_jrdd = prev._jrdd
         self.is_cached = False
+        self.is_checkpointed = False
         self.ctx = prev.ctx
         self.prev = prev
         self._jrdd_val = None
@@ -695,18 +729,21 @@ class PipelinedRDD(RDD):
         pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds)
         broadcast_vars = ListConverter().convert(
             [x._jbroadcast for x in self.ctx._pickled_broadcast_vars],
-            self.ctx.gateway._gateway_client)
+            self.ctx._gateway._gateway_client)
         self.ctx._pickled_broadcast_vars.clear()
         class_manifest = self._prev_jrdd.classManifest()
         env = copy.copy(self.ctx.environment)
         env['PYTHONPATH'] = os.environ.get("PYTHONPATH", "")
-        env = MapConverter().convert(env, self.ctx.gateway._gateway_client)
-        python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(),
+        env = MapConverter().convert(env, self.ctx._gateway._gateway_client)
+        python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(),
             pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec,
-            broadcast_vars, class_manifest)
+            broadcast_vars, self.ctx._javaAccumulator, class_manifest)
         self._jrdd_val = python_rdd.asJavaRDD()
         return self._jrdd_val
 
+    def _is_pipelinable(self):
+        return not (self.is_cached or self.is_checkpointed)
+
 
 def _test():
     import doctest
@@ -715,8 +752,10 @@ def _test():
     # The small batch size here ensures that we see multiple batches,
     # even in these small test examples:
     globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
-    doctest.testmod(globs=globs)
+    (failure_count, test_count) = doctest.testmod(globs=globs)
     globs['sc'].stop()
+    if failure_count:
+        exit(-1)
 
 
 if __name__ == "__main__":
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 9a5151ea00341459f83b85938b82a8555a91945b..115cf28cc2ba461754530a8c3e7da33c6e7c3ed9 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -52,8 +52,13 @@ def read_int(stream):
         raise EOFError
     return struct.unpack("!i", length)[0]
 
+
+def write_int(value, stream):
+    stream.write(struct.pack("!i", value))
+
+
 def write_with_length(obj, stream):
-    stream.write(struct.pack("!i", len(obj)))
+    write_int(len(obj), stream)
     stream.write(obj)
 
 
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index 7e6ad3aa7604672884b8458deabdac8888717eee..54ff1bf8e7c3ea63db72a5bd9015e26705948b9d 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -1,9 +1,10 @@
 """
 An interactive shell.
 
-This fle is designed to be launched as a PYTHONSTARTUP script.
+This file is designed to be launched as a PYTHONSTARTUP script.
 """
 import os
+import pyspark
 from pyspark.context import SparkContext
 
 
@@ -14,4 +15,4 @@ print "Spark context avaiable as sc."
 # which allows us to execute the user's PYTHONSTARTUP file:
 _pythonstartup = os.environ.get('OLD_PYTHONSTARTUP')
 if _pythonstartup and os.path.isfile(_pythonstartup):
-        execfile(_pythonstartup)
+    execfile(_pythonstartup)
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
new file mode 100644
index 0000000000000000000000000000000000000000..6a1962d26795dfe950049de5b3a07b3d7f12354a
--- /dev/null
+++ b/python/pyspark/tests.py
@@ -0,0 +1,121 @@
+"""
+Unit tests for PySpark; additional tests are implemented as doctests in
+individual modules.
+"""
+import os
+import shutil
+import sys
+from tempfile import NamedTemporaryFile
+import time
+import unittest
+
+from pyspark.context import SparkContext
+from pyspark.files import SparkFiles
+from pyspark.java_gateway import SPARK_HOME
+
+
+class PySparkTestCase(unittest.TestCase):
+
+    def setUp(self):
+        self._old_sys_path = list(sys.path)
+        class_name = self.__class__.__name__
+        self.sc = SparkContext('local[4]', class_name , batchSize=2)
+
+    def tearDown(self):
+        self.sc.stop()
+        sys.path = self._old_sys_path
+        # To avoid Akka rebinding to the same port, since it doesn't unbind
+        # immediately on shutdown
+        self.sc._jvm.System.clearProperty("spark.driver.port")
+
+
+class TestCheckpoint(PySparkTestCase):
+
+    def setUp(self):
+        PySparkTestCase.setUp(self)
+        self.checkpointDir = NamedTemporaryFile(delete=False)
+        os.unlink(self.checkpointDir.name)
+        self.sc.setCheckpointDir(self.checkpointDir.name)
+
+    def tearDown(self):
+        PySparkTestCase.tearDown(self)
+        shutil.rmtree(self.checkpointDir.name)
+
+    def test_basic_checkpointing(self):
+        parCollection = self.sc.parallelize([1, 2, 3, 4])
+        flatMappedRDD = parCollection.flatMap(lambda x: range(1, x + 1))
+
+        self.assertFalse(flatMappedRDD.isCheckpointed())
+        self.assertIsNone(flatMappedRDD.getCheckpointFile())
+
+        flatMappedRDD.checkpoint()
+        result = flatMappedRDD.collect()
+        time.sleep(1)  # 1 second
+        self.assertTrue(flatMappedRDD.isCheckpointed())
+        self.assertEqual(flatMappedRDD.collect(), result)
+        self.assertEqual(self.checkpointDir.name,
+                         os.path.dirname(flatMappedRDD.getCheckpointFile()))
+
+    def test_checkpoint_and_restore(self):
+        parCollection = self.sc.parallelize([1, 2, 3, 4])
+        flatMappedRDD = parCollection.flatMap(lambda x: [x])
+
+        self.assertFalse(flatMappedRDD.isCheckpointed())
+        self.assertIsNone(flatMappedRDD.getCheckpointFile())
+
+        flatMappedRDD.checkpoint()
+        flatMappedRDD.count()  # forces a checkpoint to be computed
+        time.sleep(1)  # 1 second
+
+        self.assertIsNotNone(flatMappedRDD.getCheckpointFile())
+        recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile())
+        self.assertEquals([1, 2, 3, 4], recovered.collect())
+
+
+class TestAddFile(PySparkTestCase):
+
+    def test_add_py_file(self):
+        # To ensure that we're actually testing addPyFile's effects, check that
+        # this job fails due to `userlibrary` not being on the Python path:
+        def func(x):
+            from userlibrary import UserClass
+            return UserClass().hello()
+        self.assertRaises(Exception,
+                          self.sc.parallelize(range(2)).map(func).first)
+        # Add the file, so the job should now succeed:
+        path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
+        self.sc.addPyFile(path)
+        res = self.sc.parallelize(range(2)).map(func).first()
+        self.assertEqual("Hello World!", res)
+
+    def test_add_file_locally(self):
+        path = os.path.join(SPARK_HOME, "python/test_support/hello.txt")
+        self.sc.addFile(path)
+        download_path = SparkFiles.get("hello.txt")
+        self.assertNotEqual(path, download_path)
+        with open(download_path) as test_file:
+            self.assertEquals("Hello World!\n", test_file.readline())
+
+    def test_add_py_file_locally(self):
+        # To ensure that we're actually testing addPyFile's effects, check that
+        # this fails due to `userlibrary` not being on the Python path:
+        def func():
+            from userlibrary import UserClass
+        self.assertRaises(ImportError, func)
+        path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py")
+        self.sc.addFile(path)
+        from userlibrary import UserClass
+        self.assertEqual("Hello World!", UserClass().hello())
+
+
+class TestIO(PySparkTestCase):
+
+    def test_stdout_redirection(self):
+        import subprocess
+        def func(x):
+            subprocess.check_call('ls', shell=True)
+        self.sc.parallelize([1]).foreach(func)
+
+
+if __name__ == "__main__":
+    unittest.main()
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index 3d792bbaa2261a0a00313088c4fc90a316a2648d..812e7a9da5a030875cf8d632c7e2040285030599 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -1,19 +1,23 @@
 """
 Worker that receives input from Piped RDD.
 """
+import os
 import sys
+import traceback
 from base64 import standard_b64decode
 # CloudPickler needs to be imported so that depicklers are registered using the
 # copy_reg module.
+from pyspark.accumulators import _accumulatorRegistry
 from pyspark.broadcast import Broadcast, _broadcastRegistry
 from pyspark.cloudpickle import CloudPickler
-from pyspark.serializers import write_with_length, read_with_length, \
+from pyspark.files import SparkFiles
+from pyspark.serializers import write_with_length, read_with_length, write_int, \
     read_long, read_int, dump_pickle, load_pickle, read_from_pickle_file
 
 
 # Redirect stdout to stderr so that users must return values from functions.
-old_stdout = sys.stdout
-sys.stdout = sys.stderr
+old_stdout = os.fdopen(os.dup(1), 'w')
+os.dup2(2, 1)
 
 
 def load_obj():
@@ -22,6 +26,10 @@ def load_obj():
 
 def main():
     split_index = read_int(sys.stdin)
+    spark_files_dir = load_pickle(read_with_length(sys.stdin))
+    SparkFiles._root_directory = spark_files_dir
+    SparkFiles._is_running_on_worker = True
+    sys.path.append(spark_files_dir)
     num_broadcast_variables = read_int(sys.stdin)
     for _ in range(num_broadcast_variables):
         bid = read_long(sys.stdin)
@@ -34,8 +42,17 @@ def main():
     else:
         dumps = dump_pickle
     iterator = read_from_pickle_file(sys.stdin)
-    for obj in func(split_index, iterator):
-        write_with_length(dumps(obj), old_stdout)
+    try:
+        for obj in func(split_index, iterator):
+           write_with_length(dumps(obj), old_stdout)
+    except Exception as e:
+        write_int(-2, old_stdout)
+        write_with_length(traceback.format_exc(), old_stdout)
+        sys.exit(-1)
+    # Mark the beginning of the accumulators section of the output
+    write_int(-1, old_stdout)
+    for aid, accum in _accumulatorRegistry.items():
+        write_with_length(dump_pickle((aid, accum._value)), old_stdout)
 
 
 if __name__ == '__main__':
diff --git a/python/run-tests b/python/run-tests
index fcdd1e27a7f7d7082a50b670dbbcbd9e69007288..a3a9ff5dcb260a5fd813da6c989380e3673a003d 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -8,9 +8,18 @@ FAILED=0
 $FWDIR/pyspark pyspark/rdd.py
 FAILED=$(($?||$FAILED))
 
+$FWDIR/pyspark pyspark/context.py
+FAILED=$(($?||$FAILED))
+
 $FWDIR/pyspark -m doctest pyspark/broadcast.py
 FAILED=$(($?||$FAILED))
 
+$FWDIR/pyspark -m doctest pyspark/accumulators.py
+FAILED=$(($?||$FAILED))
+
+$FWDIR/pyspark -m unittest pyspark.tests
+FAILED=$(($?||$FAILED))
+
 if [[ $FAILED != 0 ]]; then
     echo -en "\033[31m"  # Red
     echo "Had test failures; see logs."
diff --git a/python/test_support/hello.txt b/python/test_support/hello.txt
new file mode 100755
index 0000000000000000000000000000000000000000..980a0d5f19a64b4b30a87d4206aade58726b60e3
--- /dev/null
+++ b/python/test_support/hello.txt
@@ -0,0 +1 @@
+Hello World!
diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py
new file mode 100755
index 0000000000000000000000000000000000000000..5bb6f5009ffbb1bb0ca54035903ba33b2967e922
--- /dev/null
+++ b/python/test_support/userlibrary.py
@@ -0,0 +1,7 @@
+"""
+Used to test shipping of code depenencies with SparkContext.addPyFile().
+"""
+
+class UserClass(object):
+    def hello(self):
+        return "Hello World!"
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index da91c0f3ab2b1d598a9e33561e88d75a025c7812..fe526a7616c7c8a8c22f45d41e7ba1cb1e6ca477 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.0-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -70,11 +70,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-      <activation>
-        <property>
-          <name>!hadoopVersion</name>
-        </property>
-      </activation>
       <properties>
         <classifier>hadoop1</classifier>
       </properties>
@@ -115,12 +110,6 @@
     </profile>
     <profile>
       <id>hadoop2</id>
-      <activation>
-        <property>
-          <name>hadoopVersion</name>
-          <value>2</value>
-        </property>
-      </activation>
       <properties>
         <classifier>hadoop2</classifier>
       </properties>
diff --git a/repl/pom.xml b/repl/pom.xml
index 38e883c7f84b8a433c7f1d8dbbd537af5de56441..0b5e400c3d14656f9ae1934d81d33594d31fc43b 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.0-SNAPSHOT</version>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -72,11 +72,6 @@
   <profiles>
     <profile>
       <id>hadoop1</id>
-      <activation>
-        <property>
-          <name>!hadoopVersion</name>
-        </property>
-      </activation>
       <properties>
         <classifier>hadoop1</classifier>
       </properties>
@@ -101,6 +96,13 @@
           <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>
@@ -121,12 +123,6 @@
     </profile>
     <profile>
       <id>hadoop2</id>
-      <activation>
-        <property>
-          <name>hadoopVersion</name>
-          <value>2</value>
-        </property>
-      </activation>
       <properties>
         <classifier>hadoop2</classifier>
       </properties>
@@ -151,6 +147,13 @@
           <classifier>hadoop2</classifier>
           <scope>runtime</scope>
         </dependency>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-streaming</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2</classifier>
+          <scope>runtime</scope>
+        </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-core</artifactId>
@@ -161,6 +164,16 @@
           <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>
diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala
index 5a1e54c929b0722df1782c984859dd22d53e5fee..e83e40376021102edf327ae634c8ce7d99c27dfb 100644
--- a/repl/src/main/scala/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/spark/repl/SparkILoop.scala
@@ -152,7 +152,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     finally in = saved
   }
 
-  
+
   def sparkCleanUp(){
     echo("Stopping spark context.")
     intp.beQuietDuring {
@@ -222,6 +222,21 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     }
   }
 
+  /** Print a welcome message */
+  def printWelcome() {
+    echo("""Welcome to
+      ____              __
+     / __/__  ___ _____/ /__
+    _\ \/ _ \/ _ `/ __/  '_/
+   /___/ .__/\_,_/_/ /_/\_\   version 0.8.0
+      /_/
+""")
+    import Properties._
+    val welcomeMsg = "Using Scala %s (%s, Java %s)".format(
+      versionString, javaVmName, javaVersion)
+    echo(welcomeMsg)
+  }
+
   /** Show the history */
   lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") {
     override def usage = "[num]"
diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties
index 4c99e450bccccff6e67cbd38e0566553764adb2d..cfb1a390e6e3a72914e580fa18a2f6c342d63364 100644
--- a/repl/src/test/resources/log4j.properties
+++ b/repl/src/test/resources/log4j.properties
@@ -1,8 +1,8 @@
-# Set everything to be logged to the console
+# Set everything to be logged to the repl/target/unit-tests.log 
 log4j.rootCategory=INFO, file
 log4j.appender.file=org.apache.log4j.FileAppender
 log4j.appender.file.append=false
-log4j.appender.file.file=spark-tests.log
+log4j.appender.file.file=repl/target/unit-tests.log
 log4j.appender.file.layout=org.apache.log4j.PatternLayout
 log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
 
diff --git a/run b/run
index 16fb0efc6e887220dac532d3a25eccb5e5a9bf1a..4755d562a7f5dd39c0e53626f1293bdb27629a0a 100755
--- a/run
+++ b/run
@@ -1,5 +1,6 @@
 #!/bin/bash
 
+
 SCALA_VERSION=2.10
 
 # Figure out where the Scala framework is installed
@@ -13,15 +14,49 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then
   . $FWDIR/conf/spark-env.sh
 fi
 
+if [ -z "$1" ]; then
+  echo "Usage: run <spark-class> [<args>]" >&2
+  exit 1
+fi
+
+# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
+# 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+=" -Dspark.akka.logLifecycleEvents=true"
+  SPARK_JAVA_OPTS=$SPARK_DAEMON_JAVA_OPTS   # Empty by default
+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.deploy.worker.Worker')
+    SPARK_JAVA_OPTS+=" $SPARK_WORKER_OPTS"
+    ;;
+  'spark.executor.StandaloneExecutorBackend')
+    SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS"
+    ;;
+  'spark.executor.MesosExecutorBackend')
+    SPARK_JAVA_OPTS+=" $SPARK_EXECUTOR_OPTS"
+    ;;
+  'spark.repl.Main')
+    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
@@ -63,6 +98,7 @@ CORE_DIR="$FWDIR/core"
 REPL_DIR="$FWDIR/repl"
 EXAMPLES_DIR="$FWDIR/examples"
 BAGEL_DIR="$FWDIR/bagel"
+STREAMING_DIR="$FWDIR/streaming"
 PYSPARK_DIR="$FWDIR/python"
 
 # Exit if the user hasn't compiled Spark
@@ -78,24 +114,40 @@ CLASSPATH+=":$FWDIR/conf"
 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
 if [ -e "$FWDIR/lib_managed" ]; then
   CLASSPATH+=":$FWDIR/lib_managed/jars/*"
   CLASSPATH+=":$FWDIR/lib_managed/bundles/*"
 fi
 CLASSPATH+=":$REPL_DIR/lib/*"
-for jar in `find "$REPL_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
-  CLASSPATH+=":$jar"
-done
+if [ -e repl-bin/target ]; then
+  for jar in `find "repl-bin/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do
+    CLASSPATH+=":$jar"
+  done
+fi
 CLASSPATH+=":$BAGEL_DIR/target/scala-$SCALA_VERSION/classes"
 for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
   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.
+if [ -e "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar ]; then
+  # Use the JAR from the SBT build
+  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/scala-$SCALA_VERSION/spark-examples"*[0-9T].jar`
+fi
+if [ -e "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar ]; then
+  # Use the JAR from the Maven build
+  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples-"*hadoop[12].jar`
+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
diff --git a/run2.cmd b/run2.cmd
index 67f1e465e47b37d0b1a63cc6d2a22e2993462147..d2d4807971d295cd131651aab8a6de49dc0d88c6 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
@@ -11,9 +11,23 @@ set SPARK_HOME=%FWDIR%
 rem Load environment variables from conf\spark-env.cmd, if it exists
 if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
 
+rem Test that an argument was given
+if not "x%1"=="x" goto arg_given
+  echo Usage: run ^<spark-class^> [^<args^>]
+  goto exit
+:arg_given
+
+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%
+
 rem Check that SCALA_HOME has been specified
 if not "x%SCALA_HOME%"=="x" goto scala_exists
-  echo "SCALA_HOME is not set"
+  echo SCALA_HOME is not set
   goto exit
 :scala_exists
 
@@ -34,18 +48,31 @@ set CORE_DIR=%FWDIR%core
 set REPL_DIR=%FWDIR%repl
 set EXAMPLES_DIR=%FWDIR%examples
 set BAGEL_DIR=%FWDIR%bagel
+set STREAMING_DIR=%FWDIR%streaming
 set PYSPARK_DIR=%FWDIR%python
 
 rem Build up classpath
 set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes
 set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources
+set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\classes;%STREAMING_DIR%\target\scala-%SCALA_VERSION%\test-classes
+set CLASSPATH=%CLASSPATH%;%STREAMING_DIR%\lib\org\apache\kafka\kafka\0.7.2-spark\*
 set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes
-for /R "%FWDIR%\lib_managed\jars" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
-for /R "%FWDIR%\lib_managed\bundles" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
-for /R "%REPL_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
-for /R "%PYSPARK_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
+set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\*
+set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\*
+set CLASSPATH=%CLASSPATH%;%FWDIR%repl\lib\*
+set CLASSPATH=%CLASSPATH%;%FWDIR%python\lib\*
 set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes
 
+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 (
+  set SPARK_EXAMPLES_JAR=examples/target/scala-%SCALA_VERSION%/%%d
+)
+rem Then search in the build path from Maven:
+for %%d in ("examples/target/spark-examples*hadoop*.jar") do (
+  set SPARK_EXAMPLES_JAR=examples/target/%%d
+)
+
 rem Figure out whether to run our class with java or with the scala launcher.
 rem In most cases, we'd prefer to execute our process with java because scala
 rem creates a shell script as the parent of its Java process, which makes it
diff --git a/sbt/sbt b/sbt/sbt
index a3055c13c1b0fb420f901122e59f4a2cf01f3ecd..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=200m $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/sbt/sbt.cmd b/sbt/sbt.cmd
index 6b289ab44761c11846fb212984a5f022b9d52a3d..ce3ae7017458b1ee3703ad7cd0a928ebda2756af 100644
--- a/sbt/sbt.cmd
+++ b/sbt/sbt.cmd
@@ -2,4 +2,4 @@
 set EXTRA_ARGS=
 if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java
 set SPARK_HOME=%~dp0..
-java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-*.jar "%*"
+java -Xmx1200M -XX:MaxPermSize=200m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*"
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar
new file mode 100644
index 0000000000000000000000000000000000000000..65f79925a4d06a41b7b98d7e6f92fedb408c9b3a
Binary files /dev/null and b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar differ
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.md5 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.md5
new file mode 100644
index 0000000000000000000000000000000000000000..29f45f4adb6975e36cb95ee2f50de53ef76b0c5b
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.md5
@@ -0,0 +1 @@
+18876b8bc2e4cef28b6d191aa49d963f
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.sha1 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.sha1
new file mode 100644
index 0000000000000000000000000000000000000000..e3bd62bac038f6e56f7bbaf554a544f441dfbbaa
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.jar.sha1
@@ -0,0 +1 @@
+06b27270ffa52250a2c08703b397c99127b72060
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom
new file mode 100644
index 0000000000000000000000000000000000000000..082d35726a5afe5edb882bc4caac7bc6a7aafdc0
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom
@@ -0,0 +1,9 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <groupId>org.apache.kafka</groupId>
+  <artifactId>kafka</artifactId>
+  <version>0.7.2-spark</version>
+  <description>POM was created from install:install-file</description>
+</project>
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.md5 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.md5
new file mode 100644
index 0000000000000000000000000000000000000000..92c4132b5b01c48b0e17458876a7beeeed3e3084
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.md5
@@ -0,0 +1 @@
+7bc4322266e6032bdf9ef6eebdd8097d
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.sha1 b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.sha1
new file mode 100644
index 0000000000000000000000000000000000000000..8a1d8a097a113cf674588322a978235f73c0d7f4
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/0.7.2-spark/kafka-0.7.2-spark.pom.sha1
@@ -0,0 +1 @@
+d0f79e8eff0db43ca7bcf7dce2c8cd2972685c9d
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml
new file mode 100644
index 0000000000000000000000000000000000000000..720cd51c2f5e6408a63eae37c92f52210de58d55
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml
@@ -0,0 +1,12 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<metadata>
+  <groupId>org.apache.kafka</groupId>
+  <artifactId>kafka</artifactId>
+  <versioning>
+    <release>0.7.2-spark</release>
+    <versions>
+      <version>0.7.2-spark</version>
+    </versions>
+    <lastUpdated>20130121015225</lastUpdated>
+  </versioning>
+</metadata>
diff --git a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.md5 b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.md5
new file mode 100644
index 0000000000000000000000000000000000000000..a4ce5dc9e8d9b21320106941f0f8b36c3e2b6485
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.md5
@@ -0,0 +1 @@
+e2b9c7c5f6370dd1d21a0aae5e8dcd77
\ No newline at end of file
diff --git a/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.sha1 b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.sha1
new file mode 100644
index 0000000000000000000000000000000000000000..b869eaf2a61de2fbaa3f75c2e6c9df874ac92a5e
--- /dev/null
+++ b/streaming/lib/org/apache/kafka/kafka/maven-metadata-local.xml.sha1
@@ -0,0 +1 @@
+2a4341da936b6c07a09383d17ffb185ac558ee91
\ No newline at end of file
diff --git a/streaming/pom.xml b/streaming/pom.xml
new file mode 100644
index 0000000000000000000000000000000000000000..b0d0cd0ff35ee2e0c5bd63b03f0083b31ceff089
--- /dev/null
+++ b/streaming/pom.xml
@@ -0,0 +1,153 @@
+<?xml version="1.0" encoding="UTF-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>
+  <parent>
+    <groupId>org.spark-project</groupId>
+    <artifactId>spark-parent</artifactId>
+    <version>0.8.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <groupId>org.spark-project</groupId>
+  <artifactId>spark-streaming</artifactId>
+  <packaging>jar</packaging>
+  <name>Spark Project Streaming</name>
+  <url>http://spark-project.org/</url>
+
+  <repositories>
+    <!-- A repository in the local filesystem for the Kafka JAR, which we modified for Scala 2.9 -->
+    <repository>
+      <id>lib</id>
+      <url>file://${project.basedir}/lib</url>
+    </repository>
+  </repositories>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.jackson</groupId>
+      <artifactId>jackson-mapper-asl</artifactId>
+      <version>1.9.11</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.kafka</groupId>
+      <artifactId>kafka</artifactId>
+      <version>0.7.2-spark</version>  <!-- Comes from our in-project repository -->
+    </dependency>
+    <dependency>
+      <groupId>org.apache.flume</groupId>
+      <artifactId>flume-ng-sdk</artifactId>
+      <version>1.2.0</version>
+    </dependency>
+    <dependency>
+      <groupId>com.github.sgroschupf</groupId>
+      <artifactId>zkclient</artifactId>
+      <version>0.1</version>
+    </dependency>
+    <dependency>
+      <groupId>org.twitter4j</groupId>
+      <artifactId>twitter4j-stream</artifactId>
+      <version>3.0.3</version>
+    </dependency>
+    <dependency>
+      <groupId>com.typesafe.akka</groupId>
+      <artifactId>akka-zeromq</artifactId>
+      <version>2.0.3</version>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalacheck</groupId>
+      <artifactId>scalacheck_${scala.version}</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.novocode</groupId>
+      <artifactId>junit-interface</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
+    <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+      </plugin>
+    </plugins>
+  </build>
+
+  <profiles>
+    <profile>
+      <id>hadoop1</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop1</classifier>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-core</artifactId>
+          <scope>provided</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop1</classifier>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+    <profile>
+      <id>hadoop2</id>
+      <dependencies>
+        <dependency>
+          <groupId>org.spark-project</groupId>
+          <artifactId>spark-core</artifactId>
+          <version>${project.version}</version>
+          <classifier>hadoop2</classifier>
+        </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>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <configuration>
+              <classifier>hadoop2</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
new file mode 100644
index 0000000000000000000000000000000000000000..e303e33e5e4014e7b252b491edc9d5090dc5e88f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -0,0 +1,141 @@
+package spark.streaming
+
+import spark.{Logging, Utils}
+
+import org.apache.hadoop.fs.{FileUtil, Path}
+import org.apache.hadoop.conf.Configuration
+
+import java.io._
+import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
+import java.util.concurrent.Executors
+
+
+private[streaming]
+class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
+  extends Logging with Serializable {
+  val master = ssc.sc.master
+  val framework = ssc.sc.appName
+  val sparkHome = ssc.sc.sparkHome
+  val jars = ssc.sc.jars
+  val environment = ssc.sc.environment
+  val graph = ssc.graph
+  val checkpointDir = ssc.checkpointDir
+  val checkpointDuration = ssc.checkpointDuration
+  val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
+
+  def validate() {
+    assert(master != null, "Checkpoint.master is null")
+    assert(framework != null, "Checkpoint.framework is null")
+    assert(graph != null, "Checkpoint.graph is null")
+    assert(checkpointTime != null, "Checkpoint.checkpointTime is null")
+    logInfo("Checkpoint for time " + checkpointTime + " validated")
+  }
+}
+
+/**
+ * Convenience class to speed up the writing of graph checkpoint to file
+ */
+private[streaming]
+class CheckpointWriter(checkpointDir: String) extends Logging {
+  val file = new Path(checkpointDir, "graph")
+  val conf = new Configuration()
+  var fs = file.getFileSystem(conf)
+  val maxAttempts = 3
+  val executor = Executors.newFixedThreadPool(1)
+
+  class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
+    def run() {
+      var attempts = 0
+      val startTime = System.currentTimeMillis()
+      while (attempts < maxAttempts) {
+        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)
+          fos.write(bytes)
+          fos.close()
+          fos.close()
+          val finishTime = System.currentTimeMillis();
+          logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
+            "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
+          return
+        } catch {
+          case ioe: IOException =>
+            logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe)
+        }
+      }
+      logError("Could not write checkpoint for time " + checkpointTime + " to file '" + file + "'")
+    }
+  }
+
+  def write(checkpoint: Checkpoint) {
+    val bos = new ByteArrayOutputStream()
+    val zos = new LZFOutputStream(bos)
+    val oos = new ObjectOutputStream(zos)
+    oos.writeObject(checkpoint)
+    oos.close()
+    bos.close()
+    executor.execute(new CheckpointWriteHandler(checkpoint.checkpointTime, bos.toByteArray))
+  }
+
+  def stop() {
+    executor.shutdown()
+  }
+}
+
+
+private[streaming]
+object CheckpointReader extends Logging {
+
+  def read(path: String): Checkpoint = {
+    val fs = new Path(path).getFileSystem(new Configuration())
+    val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
+
+    attempts.foreach(file => {
+      if (fs.exists(file)) {
+        logInfo("Attempting to load checkpoint from file '" + file + "'")
+        try {
+          val fis = fs.open(file)
+          // ObjectInputStream uses the last defined user-defined class loader in the stack
+          // to find classes, which maybe the wrong class loader. Hence, a inherited version
+          // of ObjectInputStream is used to explicitly use the current thread's default class
+          // loader to find and load classes. This is a well know Java issue and has popped up
+          // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
+          val zis = new LZFInputStream(fis)
+          val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader)
+          val cp = ois.readObject.asInstanceOf[Checkpoint]
+          ois.close()
+          fs.close()
+          cp.validate()
+          logInfo("Checkpoint successfully loaded from file '" + file + "'")
+          logInfo("Checkpoint was generated at time " + cp.checkpointTime)
+          return cp
+        } catch {
+          case e: Exception =>
+            logError("Error loading checkpoint from file '" + file + "'", e)
+        }
+      } else {
+        logWarning("Could not read checkpoint from file '" + file + "' as it does not exist")
+      }
+
+    })
+    throw new Exception("Could not read checkpoint from path '" + path + "'")
+  }
+}
+
+private[streaming]
+class ObjectInputStreamWithLoader(inputStream_ : InputStream, loader: ClassLoader) extends ObjectInputStream(inputStream_) {
+  override def resolveClass(desc: ObjectStreamClass): Class[_] = {
+    try {
+      return loader.loadClass(desc.getName())
+    } catch {
+      case e: Exception =>
+    }
+    return super.resolveClass(desc)
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e1be5ef51cc9cd8cc4c71c52b6f8dcb820dbb761
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/DStream.scala
@@ -0,0 +1,678 @@
+package spark.streaming
+
+import spark.streaming.dstream._
+import StreamingContext._
+//import Time._
+
+import spark.{RDD, Logging}
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashMap
+
+import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.conf.Configuration
+
+/**
+ * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]]
+ * for more details on RDDs). DStreams can either be created from live data (such as, data from
+ * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
+ * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
+ * DStream periodically generates a RDD, either from live data or by transforming the RDD generated
+ * by a parent DStream.
+ *
+ * This class contains the basic operations available on all DStreams, such as `map`, `filter` and
+ * `window`. In addition, [[spark.streaming.PairDStreamFunctions]] contains operations available
+ * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`. These operations
+ * are automatically available on any DStream of the right type (e.g., DStream[(Int, Int)] through
+ * implicit conversions when `spark.streaming.StreamingContext._` is imported.
+ *
+ * DStreams internally is characterized by a few basic properties:
+ *  - A list of other DStreams that the DStream depends on
+ *  - A time interval at which the DStream generates an RDD
+ *  - A function that is used to generate an RDD after each time interval
+ */
+
+abstract class DStream[T: ClassManifest] (
+    @transient protected[streaming] var ssc: StreamingContext
+  ) extends Serializable with Logging {
+
+  initLogging()
+
+  // =======================================================================
+  // Methods that should be implemented by subclasses of DStream
+  // =======================================================================
+
+  /** Time interval after which the DStream generates a RDD */
+  def slideDuration: Duration
+
+  /** List of parent DStreams on which this DStream depends on */
+  def dependencies: List[DStream[_]]
+
+  /** Method that generates a RDD for the given time */
+  def compute (validTime: Time): Option[RDD[T]]
+
+  // =======================================================================
+  // Methods and fields available on all DStreams
+  // =======================================================================
+
+  // RDDs generated, marked as protected[streaming] so that testsuites can access it
+  @transient
+  protected[streaming] var generatedRDDs = new HashMap[Time, RDD[T]] ()
+  
+  // Time zero for the DStream
+  protected[streaming] var zeroTime: Time = null
+
+  // Duration for which the DStream will remember each RDD created
+  protected[streaming] var rememberDuration: Duration = null
+
+  // Storage level of the RDDs in the stream
+  protected[streaming] var storageLevel: StorageLevel = StorageLevel.NONE
+
+  // Checkpoint details
+  protected[streaming] val mustCheckpoint = false
+  protected[streaming] var checkpointDuration: Duration = null
+  protected[streaming] val checkpointData = new DStreamCheckpointData(this)
+
+  // Reference to whole DStream graph
+  protected[streaming] var graph: DStreamGraph = null
+
+  protected[streaming] def isInitialized = (zeroTime != null)
+
+  // Duration for which the DStream requires its parent DStream to remember each RDD created
+  protected[streaming] def parentRememberDuration = rememberDuration
+
+  /** Return the StreamingContext associated with this DStream */
+  def context = ssc
+
+  /** Persist the RDDs of this DStream with the given storage level */
+  def persist(level: StorageLevel): DStream[T] = {
+    if (this.isInitialized) {
+      throw new UnsupportedOperationException(
+        "Cannot change storage level of an DStream after streaming context has started")
+    }
+    this.storageLevel = level
+    this
+  }
+
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  def persist(): DStream[T] = persist(StorageLevel.MEMORY_ONLY_SER)
+
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  def cache(): DStream[T] = persist()
+
+  /**
+   * Enable periodic checkpointing of RDDs of this DStream
+   * @param interval Time interval after which generated RDD will be checkpointed
+   */
+  def checkpoint(interval: Duration): DStream[T] = {
+    if (isInitialized) {
+      throw new UnsupportedOperationException(
+        "Cannot change checkpoint interval of an DStream after streaming context has started")
+    }
+    persist()
+    checkpointDuration = interval
+    this
+  }
+
+  /**
+   * Initialize the DStream by setting the "zero" time, based on which
+   * the validity of future times is calculated. This method also recursively initializes
+   * its parent DStreams.
+   */
+  protected[streaming] def initialize(time: Time) {
+    if (zeroTime != null && zeroTime != time) {
+      throw new Exception("ZeroTime is already initialized to " + zeroTime
+        + ", cannot initialize it again to " + time)
+    }
+    zeroTime = time
+
+    // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger
+    if (mustCheckpoint && checkpointDuration == null) {
+      checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt
+      logInfo("Checkpoint interval automatically set to " + checkpointDuration)
+    }
+
+    // Set the minimum value of the rememberDuration if not already set
+    var minRememberDuration = slideDuration
+    if (checkpointDuration != null && minRememberDuration <= checkpointDuration) {
+      minRememberDuration = checkpointDuration * 2  // times 2 just to be sure that the latest checkpoint is not forgetten
+    }
+    if (rememberDuration == null || rememberDuration < minRememberDuration) {
+      rememberDuration = minRememberDuration
+    }
+
+    // Initialize the dependencies
+    dependencies.foreach(_.initialize(zeroTime))
+  }
+
+  protected[streaming] def validate() {
+    assert(rememberDuration != null, "Remember duration is set to null")
+
+    assert(
+      !mustCheckpoint || checkpointDuration != null,
+      "The checkpoint interval for " + this.getClass.getSimpleName + " has not been set." +
+        " Please use DStream.checkpoint() to set the interval."
+    )
+
+    assert(
+     checkpointDuration == null || context.sparkContext.checkpointDir.isDefined,
+      "The checkpoint directory has not been set. Please use StreamingContext.checkpoint()" +
+      " or SparkContext.checkpoint() to set the checkpoint directory."
+    )
+
+    assert(
+      checkpointDuration == null || checkpointDuration >= slideDuration,
+      "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " +
+        checkpointDuration + " which is lower than its slide time (" + slideDuration + "). " +
+        "Please set it to at least " + slideDuration + "."
+    )
+
+    assert(
+      checkpointDuration == null || checkpointDuration.isMultipleOf(slideDuration),
+      "The checkpoint interval for " + this.getClass.getSimpleName + " has been set to " +
+        checkpointDuration + " which not a multiple of its slide time (" + slideDuration + "). " +
+        "Please set it to a multiple " + slideDuration + "."
+    )
+
+    assert(
+      checkpointDuration == null || storageLevel != StorageLevel.NONE,
+      "" + this.getClass.getSimpleName + " has been marked for checkpointing but the storage " +
+        "level has not been set to enable persisting. Please use DStream.persist() to set the " +
+        "storage level to use memory for better checkpointing performance."
+    )
+
+    assert(
+      checkpointDuration == null || rememberDuration > checkpointDuration,
+      "The remember duration for " + this.getClass.getSimpleName + " has been set to " +
+        rememberDuration + " which is not more than the checkpoint interval (" +
+        checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
+    )
+
+    val metadataCleanerDelay = spark.util.MetadataCleaner.getDelaySeconds
+    logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
+    assert(
+      metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,
+      "It seems you are doing some DStream window operation or setting a checkpoint interval " +
+        "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " +
+        "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" +
+        "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " +
+        "set the Java property 'spark.cleaner.delay' to more than " +
+        math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds."
+    )
+
+    dependencies.foreach(_.validate())
+
+    logInfo("Slide time = " + slideDuration)
+    logInfo("Storage level = " + storageLevel)
+    logInfo("Checkpoint interval = " + checkpointDuration)
+    logInfo("Remember duration = " + rememberDuration)
+    logInfo("Initialized and validated " + this)
+  }
+
+  protected[streaming] def setContext(s: StreamingContext) {
+    if (ssc != null && ssc != s) {
+      throw new Exception("Context is already set in " + this + ", cannot set it again")
+    }
+    ssc = s
+    logInfo("Set context for " + this)
+    dependencies.foreach(_.setContext(ssc))
+  }
+
+  protected[streaming] def setGraph(g: DStreamGraph) {
+    if (graph != null && graph != g) {
+      throw new Exception("Graph is already set in " + this + ", cannot set it again")
+    }
+    graph = g
+    dependencies.foreach(_.setGraph(graph))
+  }
+
+  protected[streaming] def remember(duration: Duration) {
+    if (duration != null && duration > rememberDuration) {
+      rememberDuration = duration
+      logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this)
+    }
+    dependencies.foreach(_.remember(parentRememberDuration))
+  }
+
+  /** Checks whether the 'time' is valid wrt slideDuration for generating RDD */
+  protected def isTimeValid(time: Time): Boolean = {
+    if (!isInitialized) {
+      throw new Exception (this + " has not been initialized")
+    } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) {
+      logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime))
+      false
+    } else {
+      logInfo("Time " + time + " is valid")
+      true
+    }
+  }
+
+  /**
+   * Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal
+   * method that should not be called directly.
+   */  
+  protected[streaming] def getOrCompute(time: Time): Option[RDD[T]] = {
+    // If this DStream was not initialized (i.e., zeroTime not set), then do it
+    // If RDD was already generated, then retrieve it from HashMap
+    generatedRDDs.get(time) match {
+      
+      // If an RDD was already generated and is being reused, then 
+      // probably all RDDs in this DStream will be reused and hence should be cached
+      case Some(oldRDD) => Some(oldRDD)
+      
+      // if RDD was not generated, and if the time is valid
+      // (based on sliding time of this DStream), then generate the RDD
+      case None => {
+        if (isTimeValid(time)) {
+          compute(time) match {
+            case Some(newRDD) =>
+              if (storageLevel != StorageLevel.NONE) {
+                newRDD.persist(storageLevel)
+                logInfo("Persisting RDD " + newRDD.id + " for time " + time + " to " + storageLevel + " at time " + time)
+              }
+              if (checkpointDuration != null && (time - zeroTime).isMultipleOf(checkpointDuration)) {
+                newRDD.checkpoint()
+                logInfo("Marking RDD " + newRDD.id + " for time " + time + " for checkpointing at time " + time)
+              }
+              generatedRDDs.put(time, newRDD)
+              Some(newRDD)
+            case None => 
+              None
+          }
+        } else {
+          None
+        }
+      }
+    }
+  }
+
+  /**
+   * Generate a SparkStreaming job for the given time. This is an internal method that
+   * should not be called directly. This default implementation creates a job
+   * that materializes the corresponding RDD. Subclasses of DStream may override this
+   * to generate their own jobs.
+   */
+  protected[streaming] def generateJob(time: Time): Option[Job] = {
+    getOrCompute(time) match {
+      case Some(rdd) => {
+        val jobFunc = () => {
+          val emptyFunc = { (iterator: Iterator[T]) => {} }
+          context.sparkContext.runJob(rdd, emptyFunc)
+        }
+        Some(new Job(time, jobFunc))
+      }
+      case None => None
+    }
+  }
+
+  /**
+   * Clear metadata that are older than `rememberDuration` of this DStream.
+   * This is an internal method that should not be called directly. This default
+   * implementation clears the old generated RDDs. Subclasses of DStream may override
+   * this to clear their own metadata along with the generated RDDs.
+   */
+  protected[streaming] def clearOldMetadata(time: Time) {
+    var numForgotten = 0
+    val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration))
+    generatedRDDs --= oldRDDs.keys
+    logInfo("Cleared " + oldRDDs.size + " RDDs that were older than " +
+      (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", "))
+    dependencies.foreach(_.clearOldMetadata(time))
+  }
+
+  /* Adds metadata to the Stream while it is running. 
+   * This methd should be overwritten by sublcasses of InputDStream.
+   */
+  protected[streaming] def addMetadata(metadata: Any) {
+    if (metadata != null) {
+      logInfo("Dropping Metadata: " + metadata.toString)
+    }
+  }
+
+  /**
+   * Refresh the list of checkpointed RDDs that will be saved along with checkpoint of
+   * this stream. This is an internal method that should not be called directly. This is
+   * a default implementation that saves only the file names of the checkpointed RDDs to
+   * checkpointData. Subclasses of DStream (especially those of InputDStream) may override
+   * this method to save custom checkpoint data.
+   */
+  protected[streaming] def updateCheckpointData(currentTime: Time) {
+    logInfo("Updating checkpoint data for time " + currentTime)
+    checkpointData.update()
+    dependencies.foreach(_.updateCheckpointData(currentTime))
+    checkpointData.cleanup()
+    logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData)
+  }
+
+  /**
+   * Restore the RDDs in generatedRDDs from the checkpointData. This is an internal method
+   * that should not be called directly. This is a default implementation that recreates RDDs
+   * from the checkpoint file names stored in checkpointData. Subclasses of DStream that
+   * override the updateCheckpointData() method would also need to override this method.
+   */
+  protected[streaming] def restoreCheckpointData() {
+    // Create RDDs from the checkpoint data
+    logInfo("Restoring checkpoint data")
+    checkpointData.restore()
+    dependencies.foreach(_.restoreCheckpointData())
+    logInfo("Restored checkpoint data")
+  }
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    logDebug(this.getClass().getSimpleName + ".writeObject used")
+    if (graph != null) {
+      graph.synchronized {
+        if (graph.checkpointInProgress) {
+          oos.defaultWriteObject()
+        } else {
+          val msg = "Object of " + this.getClass.getName + " is being serialized " +
+            " possibly as a part of closure of an RDD operation. This is because " +
+            " the DStream object is being referred to from within the closure. " +
+            " Please rewrite the RDD operation inside this DStream to avoid this. " +
+            " This has been enforced to avoid bloating of Spark tasks " +
+            " with unnecessary objects."
+          throw new java.io.NotSerializableException(msg)
+        }
+      }
+    } else {
+      throw new java.io.NotSerializableException("Graph is unexpectedly null when DStream is being serialized.")
+    }
+  }
+
+  @throws(classOf[IOException])
+  private def readObject(ois: ObjectInputStream) {
+    logDebug(this.getClass().getSimpleName + ".readObject used")
+    ois.defaultReadObject()
+    generatedRDDs = new HashMap[Time, RDD[T]] ()
+  }
+
+  // =======================================================================
+  // DStream operations
+  // =======================================================================
+
+  /** Return a new DStream by applying a function to all elements of this DStream. */
+  def map[U: ClassManifest](mapFunc: T => U): DStream[U] = {
+    new MappedDStream(this, context.sparkContext.clean(mapFunc))
+  }
+
+  /**
+   * Return a new DStream by applying a function to all elements of this DStream,
+   * and then flattening the results
+   */
+  def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = {
+    new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc))
+  }
+
+  /** Return a new DStream containing only the elements that satisfy a predicate. */
+  def filter(filterFunc: T => Boolean): DStream[T] = new FilteredDStream(this, filterFunc)
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying glom() to each RDD of
+   * this DStream. Applying glom() to an RDD coalesces all elements within each partition into
+   * an array.
+   */
+  def glom(): DStream[Array[T]] = new GlommedDStream(this)
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
+   * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
+   * of the RDD.
+   */
+  def mapPartitions[U: ClassManifest](
+      mapPartFunc: Iterator[T] => Iterator[U],
+      preservePartitioning: Boolean = false
+    ): DStream[U] = {
+    new MapPartitionedDStream(this, context.sparkContext.clean(mapPartFunc), preservePartitioning)
+  }
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by reducing each RDD
+   * of this DStream.
+   */
+  def reduce(reduceFunc: (T, T) => T): DStream[T] =
+    this.map(x => (null, x)).reduceByKey(reduceFunc, 1).map(_._2)
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by counting each RDD
+   * of this DStream.
+   */
+  def count(): DStream[Long] = this.map(_ => 1L).reduce(_ + _)
+
+  /**
+   * Return a new DStream in which each RDD contains the counts of each distinct value in
+   * each RDD of this DStream. Hash partitioning is used to generate
+   * the RDDs with `numPartitions` partitions (Spark's default number of partitions if
+   * `numPartitions` not specified).
+   */
+  def countByValue(numPartitions: Int = ssc.sc.defaultParallelism): DStream[(T, Long)] =
+    this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions)
+
+  /**
+   * Apply a function to each RDD in this DStream. This is an output operator, so
+   * this DStream will be registered as an output stream and therefore materialized.
+   */
+  def foreach(foreachFunc: RDD[T] => Unit) {
+    foreach((r: RDD[T], t: Time) => foreachFunc(r))
+  }
+
+  /**
+   * Apply a function to each RDD in this DStream. This is an output operator, so
+   * this DStream will be registered as an output stream and therefore materialized.
+   */
+  def foreach(foreachFunc: (RDD[T], Time) => Unit) {
+    val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
+    ssc.registerOutputStream(newStream)
+    newStream
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of this DStream.
+   */
+  def transform[U: ClassManifest](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
+    transform((r: RDD[T], t: Time) => transformFunc(r))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of this DStream.
+   */
+  def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
+    new TransformedDStream(this, context.sparkContext.clean(transformFunc))
+  }
+
+  /**
+   * Print the first ten elements of each RDD generated in this DStream. This is an output
+   * operator, so this DStream will be registered as an output stream and there materialized.
+   */
+  def print() {
+    def foreachFunc = (rdd: RDD[T], time: Time) => {
+      val first11 = rdd.take(11)
+      println ("-------------------------------------------")
+      println ("Time: " + time)
+      println ("-------------------------------------------")
+      first11.take(10).foreach(println)
+      if (first11.size > 10) println("...")
+      println()
+    }
+    val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
+    ssc.registerOutputStream(newStream)
+  }
+
+  /**
+   * Return a new DStream in which each RDD contains all the elements in seen in a
+   * sliding window of time over this DStream. The new DStream generates RDDs with
+   * the same interval as this DStream.
+   * @param windowDuration width of the window; must be a multiple of this DStream's interval.
+   */
+  def window(windowDuration: Duration): DStream[T] = window(windowDuration, this.slideDuration)
+
+  /**
+   * Return a new DStream in which each RDD contains all the elements in seen in a
+   * sliding window of time over this DStream.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def window(windowDuration: Duration, slideDuration: Duration): DStream[T] = {
+    new WindowedDStream(this, windowDuration, slideDuration)
+  }
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by reducing all
+   * elements in a sliding window over this DStream.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByWindow(
+      reduceFunc: (T, T) => T,
+      windowDuration: Duration,
+      slideDuration: Duration
+    ): DStream[T] = {
+    this.reduce(reduceFunc).window(windowDuration, slideDuration).reduce(reduceFunc)
+  }
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by reducing all
+   * elements in a sliding window over this DStream. However, the reduction is done incrementally
+   * using the old window's reduced value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   *  This is more efficient than reduceByWindow without "inverse reduce" function.
+   *  However, it is applicable to only "invertible reduce functions".
+   * @param reduceFunc associative reduce function
+   * @param invReduceFunc inverse reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByWindow(
+      reduceFunc: (T, T) => T,
+      invReduceFunc: (T, T) => T,
+      windowDuration: Duration,
+      slideDuration: Duration
+    ): DStream[T] = {
+      this.map(x => (1, x))
+          .reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1)
+          .map(_._2)
+  }
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by counting the number
+   * of elements in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with
+   * Spark's default number of partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def countByWindow(windowDuration: Duration, slideDuration: Duration): DStream[Long] = {
+    this.map(_ => 1L).reduceByWindow(_ + _, _ - _, windowDuration, slideDuration)
+  }
+
+  /**
+   * Return a new DStream in which each RDD contains the count of distinct elements in
+   * RDDs in a sliding window over this DStream. Hash partitioning is used to generate
+   * the RDDs with `numPartitions` partitions (Spark's default number of partitions if
+   * `numPartitions` not specified).
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  number of partitions of each RDD in the new DStream.
+   */
+  def countByValueAndWindow(
+      windowDuration: Duration,
+      slideDuration: Duration,
+      numPartitions: Int = ssc.sc.defaultParallelism
+    ): DStream[(T, Long)] = {
+
+    this.map(x => (x, 1L)).reduceByKeyAndWindow(
+      (x: Long, y: Long) => x + y,
+      (x: Long, y: Long) => x - y,
+      windowDuration,
+      slideDuration,
+      numPartitions,
+      (x: (T, Long)) => x._2 != 0L
+    )
+  }
+
+  /**
+   * Return a new DStream by unifying data of another DStream with this DStream.
+   * @param that Another DStream having the same slideDuration as this DStream.
+   */
+  def union(that: DStream[T]): DStream[T] = new UnionDStream[T](Array(this, that))
+
+  /**
+   * Return all the RDDs defined by the Interval object (both end times included)
+   */
+  protected[streaming] def slice(interval: Interval): Seq[RDD[T]] = {
+    slice(interval.beginTime, interval.endTime)
+  }
+
+  /**
+   * Return all the RDDs between 'fromTime' to 'toTime' (both included)
+   */
+  def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = {
+    if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) {
+      logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")")
+    }
+    if (!(toTime - zeroTime).isMultipleOf(slideDuration)) {
+      logWarning("toTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")")
+    }
+    val alignedToTime = toTime.floor(slideDuration)
+    val alignedFromTime = fromTime.floor(slideDuration)
+
+    logInfo("Slicing from " + fromTime + " to " + toTime +
+      " (aligned to " + alignedFromTime + " and " + alignedToTime + ")")
+
+    alignedFromTime.to(alignedToTime, slideDuration).flatMap(time => {
+      if (time >= zeroTime) getOrCompute(time) else None
+    })
+  }
+
+  /**
+   * Save each RDD in this DStream as a Sequence file of serialized objects.
+   * The file name at each batch interval is generated based on `prefix` and
+   * `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsObjectFiles(prefix: String, suffix: String = "") {
+    val saveFunc = (rdd: RDD[T], time: Time) => {
+      val file = rddToFileName(prefix, suffix, time)
+      rdd.saveAsObjectFile(file)
+    }
+    this.foreach(saveFunc)
+  }
+
+  /**
+   * Save each RDD in this DStream as at text file, using string representation
+   * of elements. The file name at each batch interval is generated based on
+   * `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsTextFiles(prefix: String, suffix: String = "") {
+    val saveFunc = (rdd: RDD[T], time: Time) => {
+      val file = rddToFileName(prefix, suffix, time)
+      rdd.saveAsTextFile(file)
+    }
+    this.foreach(saveFunc)
+  }
+
+  def register() {
+    ssc.registerOutputStream(this)
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6b0fade7c64d79ca6e6ddda5dff73ca07851e1be
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/DStreamCheckpointData.scala
@@ -0,0 +1,93 @@
+package spark.streaming
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.FileSystem
+import org.apache.hadoop.conf.Configuration
+import collection.mutable.HashMap
+import spark.Logging
+
+
+
+private[streaming]
+class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T])
+  extends Serializable with Logging {
+  protected val data = new HashMap[Time, AnyRef]()
+
+  @transient private var fileSystem : FileSystem = null
+  @transient private var lastCheckpointFiles: HashMap[Time, String] = null
+
+  protected[streaming] def checkpointFiles = data.asInstanceOf[HashMap[Time, String]]
+
+  /**
+   * Updates the checkpoint data of the DStream. This gets called every time
+   * the graph checkpoint is initiated. Default implementation records the
+   * checkpoint files to which the generate RDDs of the DStream has been saved.
+   */
+  def update() {
+
+    // Get the checkpointed RDDs from the generated RDDs
+    val newCheckpointFiles = dstream.generatedRDDs.filter(_._2.getCheckpointFile.isDefined)
+                                       .map(x => (x._1, x._2.getCheckpointFile.get))
+
+    // Make a copy of the existing checkpoint data (checkpointed RDDs)
+    lastCheckpointFiles = checkpointFiles.clone()
+
+    // If the new checkpoint data has checkpoints then replace existing with the new one
+    if (newCheckpointFiles.size > 0) {
+      checkpointFiles.clear()
+      checkpointFiles ++= newCheckpointFiles
+    }
+
+    // TODO: remove this, this is just for debugging
+    newCheckpointFiles.foreach {
+      case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") }
+    }
+  }
+
+  /**
+   * Cleanup old checkpoint data. This gets called every time the graph
+   * checkpoint is initiated, but after `update` is called. Default
+   * implementation, cleans up old checkpoint files.
+   */
+  def cleanup() {
+    // If there is at least on checkpoint file in the current checkpoint files,
+    // then delete the old checkpoint files.
+    if (checkpointFiles.size > 0 && lastCheckpointFiles != null) {
+      (lastCheckpointFiles -- checkpointFiles.keySet).foreach {
+        case (time, file) => {
+          try {
+            val path = new Path(file)
+            if (fileSystem == null) {
+              fileSystem = path.getFileSystem(new Configuration())
+            }
+            fileSystem.delete(path, true)
+            logInfo("Deleted checkpoint file '" + file + "' for time " + time)
+          } catch {
+            case e: Exception =>
+              logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e)
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * Restore the checkpoint data. This gets called once when the DStream graph
+   * (along with its DStreams) are being restored from a graph checkpoint file.
+   * Default implementation restores the RDDs from their checkpoint files.
+   */
+  def restore() {
+    // Create RDDs from the checkpoint data
+    checkpointFiles.foreach {
+      case(time, file) => {
+        logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'")
+        dstream.generatedRDDs += ((time, dstream.context.sparkContext.checkpointFile[T](file)))
+      }
+    }
+  }
+
+  override def toString() = {
+    "[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]"
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
new file mode 100644
index 0000000000000000000000000000000000000000..adb7f3a24d25f6fcbd1453c2e75b56b9a22d10b4
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
@@ -0,0 +1,150 @@
+package spark.streaming
+
+import dstream.InputDStream
+import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
+import collection.mutable.ArrayBuffer
+import spark.Logging
+
+final private[streaming] class DStreamGraph extends Serializable with Logging {
+  initLogging()
+
+  private val inputStreams = new ArrayBuffer[InputDStream[_]]()
+  private val outputStreams = new ArrayBuffer[DStream[_]]()
+
+  var rememberDuration: Duration = null
+  var checkpointInProgress = false
+
+  var zeroTime: Time = null
+  var startTime: Time = null
+  var batchDuration: Duration = null
+
+  def start(time: Time) {
+    this.synchronized {
+      if (zeroTime != null) {
+        throw new Exception("DStream graph computation already started")
+      }
+      zeroTime = time
+      startTime = time
+      outputStreams.foreach(_.initialize(zeroTime))
+      outputStreams.foreach(_.remember(rememberDuration))
+      outputStreams.foreach(_.validate)
+      inputStreams.par.foreach(_.start())
+    }
+  }
+
+  def restart(time: Time) {
+    this.synchronized { startTime = time }
+  }
+
+  def stop() {
+    this.synchronized {
+      inputStreams.par.foreach(_.stop())
+    }
+  }
+
+  def setContext(ssc: StreamingContext) {
+    this.synchronized {
+      outputStreams.foreach(_.setContext(ssc))
+    }
+  }
+
+  def setBatchDuration(duration: Duration) {
+    this.synchronized {
+      if (batchDuration != null) {
+        throw new Exception("Batch duration already set as " + batchDuration +
+          ". cannot set it again.")
+      }
+    }
+    batchDuration = duration
+  }
+
+  def remember(duration: Duration) {
+    this.synchronized {
+      if (rememberDuration != null) {
+        throw new Exception("Batch duration already set as " + batchDuration +
+          ". cannot set it again.")
+      }
+      rememberDuration = duration
+    }
+  }
+
+  def addInputStream(inputStream: InputDStream[_]) {
+    this.synchronized {
+      inputStream.setGraph(this)
+      inputStreams += inputStream
+    }
+  }
+
+  def addOutputStream(outputStream: DStream[_]) {
+    this.synchronized {
+      outputStream.setGraph(this)
+      outputStreams += outputStream
+    }
+  }
+
+  def getInputStreams() = this.synchronized { inputStreams.toArray }
+
+  def getOutputStreams() = this.synchronized { outputStreams.toArray }
+
+  def generateJobs(time: Time): Seq[Job] = {
+    this.synchronized {
+      logInfo("Generating jobs for time " + time)
+      val jobs = outputStreams.flatMap(outputStream => outputStream.generateJob(time))
+      logInfo("Generated " + jobs.length + " jobs for time " + time)
+      jobs
+    }
+  }
+
+  def clearOldMetadata(time: Time) {
+    this.synchronized {
+      logInfo("Clearing old metadata for time " + time)
+      outputStreams.foreach(_.clearOldMetadata(time))
+      logInfo("Cleared old metadata for time " + time)
+    }
+  }
+
+  def updateCheckpointData(time: Time) {
+    this.synchronized {
+      logInfo("Updating checkpoint data for time " + time)
+      outputStreams.foreach(_.updateCheckpointData(time))
+      logInfo("Updated checkpoint data for time " + time)
+    }
+  }
+
+  def restoreCheckpointData() {
+    this.synchronized {
+      logInfo("Restoring checkpoint data")
+      outputStreams.foreach(_.restoreCheckpointData())
+      logInfo("Restored checkpoint data")
+    }
+  }
+
+  def validate() {
+    this.synchronized {
+      assert(batchDuration != null, "Batch duration has not been set")
+      //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + " is very low")
+      assert(getOutputStreams().size > 0, "No output streams registered, so nothing to execute")
+    }
+  }
+
+  @throws(classOf[IOException])
+  private def writeObject(oos: ObjectOutputStream) {
+    this.synchronized {
+      logDebug("DStreamGraph.writeObject used")
+      checkpointInProgress = true
+      oos.defaultWriteObject()
+      checkpointInProgress = false
+    }
+  }
+
+  @throws(classOf[IOException])
+  private def readObject(ois: ObjectInputStream) {
+    this.synchronized {
+      logDebug("DStreamGraph.readObject used")
+      checkpointInProgress = true
+      ois.defaultReadObject()
+      checkpointInProgress = false
+    }
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ee26206e249a8abf6bf9f502b7dac6eaef65e0e6
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Duration.scala
@@ -0,0 +1,62 @@
+package spark.streaming
+
+case class Duration (private val millis: Long) {
+
+  def < (that: Duration): Boolean = (this.millis < that.millis)
+
+  def <= (that: Duration): Boolean = (this.millis <= that.millis)
+
+  def > (that: Duration): Boolean = (this.millis > that.millis)
+
+  def >= (that: Duration): Boolean = (this.millis >= that.millis)
+
+  def + (that: Duration): Duration = new Duration(millis + that.millis)
+
+  def - (that: Duration): Duration = new Duration(millis - that.millis)
+
+  def * (times: Int): Duration = new Duration(millis * times)
+
+  def / (that: Duration): Double = millis.toDouble / that.millis.toDouble
+
+  def isMultipleOf(that: Duration): Boolean =
+    (this.millis % that.millis == 0)
+
+  def min(that: Duration): Duration = if (this < that) this else that
+
+  def max(that: Duration): Duration = if (this > that) this else that
+
+  def isZero: Boolean = (this.millis == 0)
+
+  override def toString: String = (millis.toString + " ms")
+
+  def toFormattedString: String = millis.toString
+
+  def milliseconds: Long = millis
+}
+
+
+/**
+ * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * a given number of milliseconds.
+ */
+object Milliseconds {
+  def apply(milliseconds: Long) = new Duration(milliseconds)
+}
+
+/**
+ * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * a given number of seconds.
+ */
+object Seconds {
+  def apply(seconds: Long) = new Duration(seconds * 1000)
+}
+
+/**
+ * Helper object that creates instance of [[spark.streaming.Duration]] representing
+ * a given number of minutes.
+ */
+object Minutes {
+  def apply(minutes: Long) = new Duration(minutes * 60000)
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6a8b81760e35b5fe81dc169155d7678c107bba9f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Interval.scala
@@ -0,0 +1,42 @@
+package spark.streaming
+
+private[streaming]
+class Interval(val beginTime: Time, val endTime: Time) {
+  def this(beginMs: Long, endMs: Long) = this(new Time(beginMs), new Time(endMs))
+  
+  def duration(): Duration = endTime - beginTime
+
+  def + (time: Duration): Interval = {
+    new Interval(beginTime + time, endTime + time) 
+  }
+
+  def - (time: Duration): Interval = {
+    new Interval(beginTime - time, endTime - time)
+  }
+
+  def < (that: Interval): Boolean = {
+    if (this.duration != that.duration) {
+      throw new Exception("Comparing two intervals with different durations [" + this + ", " + that + "]")
+    }
+    this.endTime < that.endTime
+  }
+
+  def <= (that: Interval) = (this < that || this == that)
+ 
+  def > (that: Interval) = !(this <= that)
+  
+  def >= (that: Interval) = !(this < that)
+
+  override def toString = "[" + beginTime + ", " + endTime + "]"
+}
+
+private[streaming]
+object Interval {
+  def currentInterval(duration: Duration): Interval  = {
+    val time = new Time(System.currentTimeMillis)
+    val intervalBegin = time.floor(duration)
+    new Interval(intervalBegin, intervalBegin + duration)
+  }
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/Job.scala b/streaming/src/main/scala/spark/streaming/Job.scala
new file mode 100644
index 0000000000000000000000000000000000000000..67bd8388bc8d2c3576a93ff2c02ce5b517aced09
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Job.scala
@@ -0,0 +1,24 @@
+package spark.streaming
+
+import java.util.concurrent.atomic.AtomicLong
+
+private[streaming]
+class Job(val time: Time, func: () => _) {
+  val id = Job.getNewId()
+  def run(): Long = {
+    val startTime = System.currentTimeMillis 
+    func() 
+    val stopTime = System.currentTimeMillis
+    (stopTime - startTime)
+  }
+
+  override def toString = "streaming job " + id + " @ " + time 
+}
+
+private[streaming]
+object Job {
+  val id = new AtomicLong(0)
+
+  def getNewId() = id.getAndIncrement()
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7696c4a592bf6c3c9f5885da1e779f39f7ff3101
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/JobManager.scala
@@ -0,0 +1,71 @@
+package spark.streaming
+
+import spark.Logging 
+import spark.SparkEnv
+import java.util.concurrent.Executors
+import collection.mutable.HashMap
+import collection.mutable.ArrayBuffer
+
+
+private[streaming]
+class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging {
+  
+  class JobHandler(ssc: StreamingContext, job: Job) extends Runnable {
+    def run() {
+      SparkEnv.set(ssc.env)
+      try {
+        val timeTaken = job.run()
+        logInfo("Total delay: %.5f s for job %s of time %s (execution: %.5f s)".format(
+          (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, job.time.milliseconds, timeTaken / 1000.0))
+      } catch {
+        case e: Exception =>
+          logError("Running " + job + " failed", e)
+      }
+      clearJob(job)
+    }
+  }
+
+  initLogging()
+
+  val jobExecutor = Executors.newFixedThreadPool(numThreads) 
+  val jobs = new HashMap[Time, ArrayBuffer[Job]]
+
+  def runJob(job: Job) {
+    jobs.synchronized {
+      jobs.getOrElseUpdate(job.time, new ArrayBuffer[Job]) += job
+    }
+    jobExecutor.execute(new JobHandler(ssc, job))
+    logInfo("Added " + job + " to queue")
+  }
+
+  def stop() {
+    jobExecutor.shutdown()
+  }
+
+  private def clearJob(job: Job) {
+    var timeCleared = false
+    val time = job.time
+    jobs.synchronized {
+      val jobsOfTime = jobs.get(time)
+      if (jobsOfTime.isDefined) {
+        jobsOfTime.get -= job
+        if (jobsOfTime.get.isEmpty) {
+          jobs -= time
+          timeCleared = true
+        }
+      } else {
+        throw new Exception("Job finished for time " + job.time +
+          " but time does not exist in jobs")
+      }
+    }
+    if (timeCleared) {
+      ssc.scheduler.clearOldMetadata(time)
+    }
+  }
+
+  def getPendingTimes(): Array[Time] = {
+    jobs.synchronized {
+      jobs.keySet.toArray
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b159d26c02b2d4383253110d1d589fc2a6b4261b
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
@@ -0,0 +1,156 @@
+package spark.streaming
+
+import spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}
+import spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError}
+import spark.Logging
+import spark.SparkEnv
+import spark.SparkContext._
+
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Queue
+
+import akka.actor._
+import akka.pattern.ask
+import akka.util.duration._
+import akka.dispatch._
+
+private[streaming] sealed trait NetworkInputTrackerMessage
+private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage
+private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[String], metadata: Any) extends NetworkInputTrackerMessage
+private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage
+
+/**
+ * This class manages the execution of the receivers of NetworkInputDStreams.
+ */
+private[streaming]
+class NetworkInputTracker(
+    @transient ssc: StreamingContext,
+    @transient networkInputStreams: Array[NetworkInputDStream[_]])
+  extends Logging {
+
+  val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*)
+  val receiverExecutor = new ReceiverExecutor()
+  val receiverInfo = new HashMap[Int, ActorRef]
+  val receivedBlockIds = new HashMap[Int, Queue[String]]
+  val timeout = 5000.milliseconds
+
+  var currentTime: Time = null
+
+  /** Start the actor and receiver execution thread. */
+  def start() {
+    ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), "NetworkInputTracker")
+    receiverExecutor.start()
+  }
+
+  /** Stop the receiver execution thread. */
+  def stop() {
+    // TODO: stop the actor as well
+    receiverExecutor.interrupt()
+    receiverExecutor.stopReceivers()
+  }
+
+  /** Return all the blocks received from a receiver. */
+  def getBlockIds(receiverId: Int, time: Time): Array[String] = synchronized {
+    val queue =  receivedBlockIds.synchronized {
+      receivedBlockIds.getOrElse(receiverId, new Queue[String]())
+    }
+    val result = queue.synchronized {
+      queue.dequeueAll(x => true)
+    }
+    logInfo("Stream " + receiverId + " received " + result.size + " blocks")
+    result.toArray
+  }
+
+  /** Actor to receive messages from the receivers. */
+  private class NetworkInputTrackerActor extends Actor {
+    def receive = {
+      case RegisterReceiver(streamId, receiverActor) => {
+        if (!networkInputStreamMap.contains(streamId)) {
+          throw new Exception("Register received for unexpected id " + streamId)
+        }
+        receiverInfo += ((streamId, receiverActor))
+        logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address)
+        sender ! true
+      }
+      case AddBlocks(streamId, blockIds, metadata) => {
+        val tmp = receivedBlockIds.synchronized {
+          if (!receivedBlockIds.contains(streamId)) {
+            receivedBlockIds += ((streamId, new Queue[String]))
+          }
+          receivedBlockIds(streamId)
+        }
+        tmp.synchronized {
+          tmp ++= blockIds
+        }
+        networkInputStreamMap(streamId).addMetadata(metadata)
+      }
+      case DeregisterReceiver(streamId, msg) => {
+        receiverInfo -= streamId
+        logError("De-registered receiver for network stream " + streamId
+          + " with message " + msg)
+        //TODO: Do something about the corresponding NetworkInputDStream
+      }
+    }
+  }
+
+  /** This thread class runs all the receivers on the cluster.  */
+  class ReceiverExecutor extends Thread {
+    val env = ssc.env
+
+    override def run() {
+      try {
+        SparkEnv.set(env)
+        startReceivers()
+      } catch {
+        case ie: InterruptedException => logInfo("ReceiverExecutor interrupted")
+      } finally {
+        stopReceivers()
+      }
+    }
+
+    /**
+     * Get the receivers from the NetworkInputDStreams, distributes them to the
+     * worker nodes as a parallel collection, and runs them.
+     */
+    def startReceivers() {
+      val receivers = networkInputStreams.map(nis => {
+        val rcvr = nis.getReceiver()
+        rcvr.setStreamId(nis.id)
+        rcvr
+      })
+
+      // Right now, we only honor preferences if all receivers have them
+      val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined).reduce(_ && _)
+
+      // Create the parallel collection of receivers to distributed them on the worker nodes
+      val tempRDD =
+        if (hasLocationPreferences) {
+          val receiversWithPreferences = receivers.map(r => (r, Seq(r.getLocationPreference().toString)))
+          ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences)
+        }
+        else {
+          ssc.sc.makeRDD(receivers, receivers.size)
+        }
+
+      // Function to start the receiver on the worker node
+      val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => {
+        if (!iterator.hasNext) {
+          throw new Exception("Could not start receiver as details not found.")
+        }
+        iterator.next().start()
+      }
+      // Run the dummy Spark job to ensure that all slaves have registered.
+      // This avoids all the receivers to be scheduled on the same node.
+      ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+
+      // Distribute the receivers and start them
+      ssc.sparkContext.runJob(tempRDD, startReceiver)
+    }
+
+    /** Stops the receivers. */
+    def stopReceivers() {
+      // Signal the receivers to stop
+      receiverInfo.values.foreach(_ ! StopReceiver)
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3ec922957d635135f21f279005405ec556c9d898
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
@@ -0,0 +1,516 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
+import spark.streaming.dstream.{CoGroupedDStream, ShuffledDStream}
+import spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
+
+import spark.{Manifests, RDD, Partitioner, HashPartitioner}
+import spark.SparkContext._
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.mapred.{JobConf, OutputFormat}
+import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
+import org.apache.hadoop.mapred.OutputFormat
+import org.apache.hadoop.conf.Configuration
+
+class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)])
+extends Serializable {
+
+  private[streaming] def ssc = self.ssc
+
+  private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = {
+    new HashPartitioner(numPartitions)
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * generate the RDDs with Spark's default number of partitions.
+   */
+  def groupByKey(): DStream[(K, Seq[V])] = {
+    groupByKey(defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * generate the RDDs with `numPartitions` partitions.
+   */
+  def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = {
+    groupByKey(defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]]
+   * is used to control the partitioning of each RDD.
+   */
+  def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = {
+    val createCombiner = (v: V) => ArrayBuffer[V](v)
+    val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v)
+    val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2)
+    combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner)
+      .asInstanceOf[DStream[(K, Seq[V])]]
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * merged using the associative reduce function. Hash partitioning is used to generate the RDDs
+   * with Spark's default number of partitions.
+   */
+  def reduceByKey(reduceFunc: (V, V) => V): DStream[(K, V)] = {
+    reduceByKey(reduceFunc, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs
+   * with `numPartitions` partitions.
+   */
+  def reduceByKey(reduceFunc: (V, V) => V, numPartitions: Int): DStream[(K, V)] = {
+    reduceByKey(reduceFunc, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the
+   * partitioning of each RDD.
+   */
+  def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = {
+    val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
+    combineByKey((v: V) => v, cleanedReduceFunc, cleanedReduceFunc, partitioner)
+  }
+
+  /**
+   * Combine elements of each key in DStream's RDDs using custom functions. This is similar to the
+   * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
+   * information.
+   */
+  def combineByKey[C: ClassManifest](
+    createCombiner: V => C,
+    mergeValue: (C, V) => C,
+    mergeCombiner: (C, C) => C,
+    partitioner: Partitioner) : DStream[(K, C)] = {
+    new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to
+   * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
+   * with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
+   * Spark's default number of partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   */
+  def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = {
+    groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window. Similar to
+   * `DStream.groupByKey()`, but applies it over a sliding window. Hash partitioning is used to
+   * generate the RDDs with Spark's default number of partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = {
+    groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  number of partitions of each RDD in the new DStream; if not specified
+   *                       then Spark's default number of partitions will be used
+   */
+  def groupByKeyAndWindow(
+      windowDuration: Duration,
+      slideDuration: Duration,
+      numPartitions: Int
+    ): DStream[(K, Seq[V])] = {
+    groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param partitioner    partitioner for controlling the partitioning of each RDD in the new DStream.
+   */
+  def groupByKeyAndWindow(
+      windowDuration: Duration,
+      slideDuration: Duration,
+      partitioner: Partitioner
+    ): DStream[(K, Seq[V])] = {
+    self.window(windowDuration, slideDuration).groupByKey(partitioner)
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+   * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream
+   * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate
+   * the RDDs with Spark's default number of partitions.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: (V, V) => V,
+      windowDuration: Duration
+    ): DStream[(K, V)] = {
+    reduceByKeyAndWindow(reduceFunc, windowDuration, self.slideDuration, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+   * generate the RDDs with Spark's default number of partitions.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: (V, V) => V, 
+      windowDuration: Duration,
+      slideDuration: Duration
+    ): DStream[(K, V)] = {
+    reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, defaultPartitioner())
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+   * generate the RDDs with `numPartitions` partitions.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  number of partitions of each RDD in the new DStream.
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: (V, V) => V,
+      windowDuration: Duration,
+      slideDuration: Duration,
+      numPartitions: Int
+    ): DStream[(K, V)] = {
+    reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to
+   * `DStream.reduceByKey()`, but applies it over a sliding window.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param partitioner    partitioner for controlling the partitioning of each RDD
+   *                       in the new DStream.
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: (V, V) => V,
+      windowDuration: Duration,
+      slideDuration: Duration,
+      partitioner: Partitioner
+    ): DStream[(K, V)] = {
+    val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
+    self.reduceByKey(cleanedReduceFunc, partitioner)
+        .window(windowDuration, slideDuration)
+        .reduceByKey(cleanedReduceFunc, partitioner)
+  }
+
+  /**
+   * Return a new DStream by applying incremental `reduceByKey` over a sliding window.
+   * The reduced value of over a new window is calculated using the old window's reduced value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   *
+   * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function.
+   * However, it is applicable to only "invertible reduce functions".
+   * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+   * @param reduceFunc associative reduce function
+   * @param invReduceFunc inverse reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param filterFunc     Optional function to filter expired key-value pairs;
+   *                       only pairs that satisfy the function are retained
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: (V, V) => V,
+      invReduceFunc: (V, V) => V,
+      windowDuration: Duration,
+      slideDuration: Duration = self.slideDuration,
+      numPartitions: Int = ssc.sc.defaultParallelism,
+      filterFunc: ((K, V)) => Boolean = null
+    ): DStream[(K, V)] = {
+
+    reduceByKeyAndWindow(
+      reduceFunc, invReduceFunc, windowDuration,
+      slideDuration, defaultPartitioner(numPartitions), filterFunc
+    )
+  }
+
+  /**
+   * Return a new DStream by applying incremental `reduceByKey` over a sliding window.
+   * The reduced value of over a new window is calculated using the old window's reduced value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   * This is more efficient than reduceByKeyAndWindow without "inverse reduce" function.
+   * However, it is applicable to only "invertible reduce functions".
+   * @param reduceFunc     associative reduce function
+   * @param invReduceFunc  inverse reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param partitioner    partitioner for controlling the partitioning of each RDD in the new DStream.
+   * @param filterFunc     Optional function to filter expired key-value pairs;
+   *                       only pairs that satisfy the function are retained
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: (V, V) => V,
+      invReduceFunc: (V, V) => V,
+      windowDuration: Duration,
+      slideDuration: Duration,
+      partitioner: Partitioner,
+      filterFunc: ((K, V)) => Boolean
+    ): DStream[(K, V)] = {
+
+    val cleanedReduceFunc = ssc.sc.clean(reduceFunc)
+    val cleanedInvReduceFunc = ssc.sc.clean(invReduceFunc)
+    val cleanedFilterFunc = if (filterFunc != null) Some(ssc.sc.clean(filterFunc)) else None
+    new ReducedWindowedDStream[K, V](
+      self, cleanedReduceFunc, cleanedInvReduceFunc, cleanedFilterFunc,
+      windowDuration, slideDuration, partitioner
+    )
+  }
+
+  /**
+   * Return a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of each key.
+   * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated.
+   * @tparam S State type
+   */
+  def updateStateByKey[S: ClassManifest](
+      updateFunc: (Seq[V], Option[S]) => Option[S]
+    ): DStream[(K, S)] = {
+    updateStateByKey(updateFunc, defaultPartitioner())
+  }
+
+  /**
+   * Return a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of each key.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated.
+   * @param numPartitions Number of partitions of each RDD in the new DStream.
+   * @tparam S State type
+   */
+  def updateStateByKey[S: ClassManifest](
+      updateFunc: (Seq[V], Option[S]) => Option[S],
+      numPartitions: Int
+    ): DStream[(K, S)] = {
+    updateStateByKey(updateFunc, defaultPartitioner(numPartitions))
+  }
+
+  /**
+   * Create a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of the key.
+   * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated.
+   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+   * @tparam S State type
+   */
+  def updateStateByKey[S: ClassManifest](
+      updateFunc: (Seq[V], Option[S]) => Option[S],
+      partitioner: Partitioner
+    ): DStream[(K, S)] = {
+    val newUpdateFunc = (iterator: Iterator[(K, Seq[V], Option[S])]) => {
+      iterator.flatMap(t => updateFunc(t._2, t._3).map(s => (t._1, s)))
+    }
+    updateStateByKey(newUpdateFunc, partitioner, true)
+  }
+
+  /**
+   * Return a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of each key.
+   * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated. Note, that
+   *                   this function may generate a different a tuple with a different key
+   *                   than the input key. It is up to the developer to decide whether to
+   *                   remember the partitioner despite the key being changed.
+   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+   * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs.
+   * @tparam S State type
+   */
+  def updateStateByKey[S: ClassManifest](
+      updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
+      partitioner: Partitioner,
+      rememberPartitioner: Boolean
+    ): DStream[(K, S)] = {
+     new StateDStream(self, ssc.sc.clean(updateFunc), partitioner, rememberPartitioner)
+  }
+
+
+  def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = {
+    new MapValuedDStream[K, V, U](self, mapValuesFunc)
+  }
+
+  def flatMapValues[U: ClassManifest](
+      flatMapValuesFunc: V => TraversableOnce[U]
+    ): DStream[(K, U)] = {
+    new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc)
+  }
+
+  /**
+   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+   * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
+   * of partitions.
+   */
+  def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
+    cogroup(other, defaultPartitioner())
+  }
+
+  /**
+   * Cogroup `this` DStream with `other` DStream using a partitioner. For each key k in corresponding RDDs of `this`
+   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+   * key in both RDDs. Partitioner is used to partition each generated RDD.
+   */
+  def cogroup[W: ClassManifest](
+      other: DStream[(K, W)],
+      partitioner: Partitioner
+    ): DStream[(K, (Seq[V], Seq[W]))] = {
+
+    val cgd = new CoGroupedDStream[K](
+      Seq(self.asInstanceOf[DStream[(K, _)]], other.asInstanceOf[DStream[(K, _)]]),
+      partitioner
+    )
+    val pdfs = new PairDStreamFunctions[K, Seq[Seq[_]]](cgd)(
+      classManifest[K],
+      Manifests.seqSeqManifest
+    )
+    pdfs.mapValues {
+      case Seq(vs, ws) =>
+        (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
+    }
+  }
+
+  /**
+   * Join `this` DStream with `other` DStream. HashPartitioner is used
+   * to partition each generated RDD into default number of partitions.
+   */
+  def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
+    join[W](other, defaultPartitioner())
+  }
+
+  /**
+   * Join `this` DStream with `other` DStream, that is, each RDD of the new DStream will
+   * be generated by joining RDDs from `this` and other DStream. Uses the given
+   * Partitioner to partition each generated RDD.
+   */
+  def join[W: ClassManifest](
+      other: DStream[(K, W)],
+      partitioner: Partitioner
+    ): DStream[(K, (V, W))] = {
+    this.cogroup(other, partitioner)
+        .flatMapValues{
+      case (vs, ws) =>
+        for (v <- vs.iterator; w <- ws.iterator) yield (v, w)
+    }
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+   * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+   */
+  def saveAsHadoopFiles[F <: OutputFormat[K, V]](
+      prefix: String,
+      suffix: String
+    )(implicit fm: ClassManifest[F]) {
+    saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is generated
+   * based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix"
+   */
+  def saveAsHadoopFiles(
+      prefix: String,
+      suffix: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: OutputFormat[_, _]],
+      conf: JobConf = new JobConf
+    ) {
+    val saveFunc = (rdd: RDD[(K, V)], time: Time) => {
+      val file = rddToFileName(prefix, suffix, time)
+      rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf)
+    }
+    self.foreach(saveFunc)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](
+      prefix: String,
+      suffix: String
+    )(implicit fm: ClassManifest[F])  {
+    saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsNewAPIHadoopFiles(
+      prefix: String,
+      suffix: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
+      conf: Configuration = new Configuration
+    ) {
+    val saveFunc = (rdd: RDD[(K, V)], time: Time) => {
+      val file = rddToFileName(prefix, suffix, time)
+      rdd.saveAsNewAPIHadoopFile(file, keyClass, valueClass, outputFormatClass, conf)
+    }
+    self.foreach(saveFunc)
+  }
+
+  private def getKeyClass() = implicitly[ClassManifest[K]].erasure
+
+  private def getValueClass() = implicitly[ClassManifest[V]].erasure
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1c4b22a8981c8db8ecb2a28fcfae2e301d000627
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala
@@ -0,0 +1,113 @@
+package spark.streaming
+
+import util.{ManualClock, RecurringTimer, Clock}
+import spark.SparkEnv
+import spark.Logging
+
+private[streaming]
+class Scheduler(ssc: StreamingContext) extends Logging {
+
+  initLogging()
+
+  val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
+  val jobManager = new JobManager(ssc, concurrentJobs)
+  val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
+    new CheckpointWriter(ssc.checkpointDir)
+  } else {
+    null
+  }
+
+  val clockClass = System.getProperty("spark.streaming.clock", "spark.streaming.util.SystemClock")
+  val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+  val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
+    longTime => generateJobs(new Time(longTime)))
+  val graph = ssc.graph
+  var latestTime: Time = null
+
+  def start() = synchronized {
+    if (ssc.isCheckpointPresent) {
+      restart()
+    } else {
+      startFirstTime()
+    }
+    logInfo("Scheduler started")
+  }
+  
+  def stop() = synchronized {
+    timer.stop()
+    jobManager.stop()
+    if (checkpointWriter != null) checkpointWriter.stop()
+    ssc.graph.stop()
+    logInfo("Scheduler stopped")    
+  }
+
+  private def startFirstTime() {
+    val startTime = new Time(timer.getStartTime())
+    graph.start(startTime - graph.batchDuration)
+    timer.start(startTime.milliseconds)
+    logInfo("Scheduler's timer started at " + startTime)
+  }
+
+  private def restart() {
+
+    // If manual clock is being used for testing, then
+    // either set the manual clock to the last checkpointed time,
+    // or if the property is defined set it to that time
+    if (clock.isInstanceOf[ManualClock]) {
+      val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds
+      val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
+      clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
+    }
+
+    val batchDuration = ssc.graph.batchDuration
+
+    // Batches when the master was down, that is,
+    // between the checkpoint and current restart time
+    val checkpointTime = ssc.initialCheckpoint.checkpointTime
+    val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds))
+    val downTimes = checkpointTime.until(restartTime, batchDuration)
+    logInfo("Batches during down time: " + downTimes.mkString(", "))
+
+    // Batches that were unprocessed before failure
+    val pendingTimes = ssc.initialCheckpoint.pendingTimes
+    logInfo("Batches pending processing: " + pendingTimes.mkString(", "))
+    // Reschedule jobs for these times
+    val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering)
+    logInfo("Batches to reschedule: " + timesToReschedule.mkString(", "))
+    timesToReschedule.foreach(time =>
+      graph.generateJobs(time).foreach(jobManager.runJob)
+    )
+
+    // Restart the timer
+    timer.start(restartTime.milliseconds)
+    logInfo("Scheduler's timer restarted at " + restartTime)
+  }
+
+  /** Generate jobs and perform checkpoint for the given `time`.  */
+  def generateJobs(time: Time) {
+    SparkEnv.set(ssc.env)
+    logInfo("\n-----------------------------------------------------\n")
+    graph.generateJobs(time).foreach(jobManager.runJob)
+    latestTime = time
+    doCheckpoint(time)
+  }
+
+  /**
+   * Clear old metadata assuming jobs of `time` have finished processing.
+   * And also perform checkpoint.
+   */
+  def clearOldMetadata(time: Time) {
+    ssc.graph.clearOldMetadata(time)
+    doCheckpoint(time)
+  }
+
+  /** Perform checkpoint for the give `time`. */
+  def doCheckpoint(time: Time) = synchronized {
+    if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
+      logInfo("Checkpointing graph for time " + time)
+      ssc.graph.updateCheckpointData(time)
+      checkpointWriter.write(new Checkpoint(ssc, time))
+    }
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b8b60aab43303197a06ae66a84637d23437e7472
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
@@ -0,0 +1,524 @@
+package spark.streaming
+
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
+import akka.zeromq.Subscribe
+
+import spark.streaming.dstream._
+
+import spark._
+import spark.streaming.receivers.ActorReceiver
+import spark.streaming.receivers.ReceiverSupervisorStrategy
+import spark.streaming.receivers.ZeroMQReceiver
+import spark.storage.StorageLevel
+import spark.util.MetadataCleaner
+import spark.streaming.receivers.ActorReceiver
+
+import scala.collection.mutable.Queue
+import scala.collection.Map
+
+import java.io.InputStream
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.UUID
+
+import org.apache.hadoop.io.LongWritable
+import org.apache.hadoop.io.Text
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
+import org.apache.hadoop.fs.Path
+import twitter4j.Status
+
+/**
+ * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
+ * information (such as, cluster URL and job name) to internally create a SparkContext, it provides
+ * methods used to create DStream from various input sources.
+ */
+class StreamingContext private (
+    sc_ : SparkContext,
+    cp_ : Checkpoint,
+    batchDur_ : Duration
+  ) extends Logging {
+
+  /**
+   * Create a StreamingContext using an existing SparkContext.
+   * @param sparkContext Existing SparkContext
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(sparkContext: SparkContext, batchDuration: Duration) = {
+    this(sparkContext, null, batchDuration)
+  }
+
+  /**
+   * Create a StreamingContext by providing the details necessary for creating a new SparkContext.
+   * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+   * @param appName A name for your job, to display on the cluster web UI
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(
+      master: String,
+      appName: String,
+      batchDuration: Duration,
+      sparkHome: String = null,
+      jars: Seq[String] = Nil,
+      environment: Map[String, String] = Map()) = {
+    this(StreamingContext.createNewSparkContext(master, appName, sparkHome, jars, environment),
+         null, batchDuration)
+  }
+
+
+  /**
+   * Re-create a StreamingContext from a checkpoint file.
+   * @param path Path either to the directory that was specified as the checkpoint directory, or
+   *             to the checkpoint file 'graph' or 'graph.bk'.
+   */
+  def this(path: String) = this(null, CheckpointReader.read(path), null)
+
+  initLogging()
+
+  if (sc_ == null && cp_ == null) {
+    throw new Exception("Spark Streaming cannot be initialized with " +
+      "both SparkContext and checkpoint as null")
+  }
+
+  if (MetadataCleaner.getDelaySeconds < 0) {
+    throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; "
+      + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)")
+  }
+
+  protected[streaming] val isCheckpointPresent = (cp_ != null)
+
+  protected[streaming] val sc: SparkContext = {
+    if (isCheckpointPresent) {
+      new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment)
+    } else {
+      sc_
+    }
+  }
+
+  protected[streaming] val env = SparkEnv.get
+
+  protected[streaming] val graph: DStreamGraph = {
+    if (isCheckpointPresent) {
+      cp_.graph.setContext(this)
+      cp_.graph.restoreCheckpointData()
+      cp_.graph
+    } else {
+      assert(batchDur_ != null, "Batch duration for streaming context cannot be null")
+      val newGraph = new DStreamGraph()
+      newGraph.setBatchDuration(batchDur_)
+      newGraph
+    }
+  }
+
+  protected[streaming] val nextNetworkInputStreamId = new AtomicInteger(0)
+  protected[streaming] var networkInputTracker: NetworkInputTracker = null
+
+  protected[streaming] var checkpointDir: String = {
+    if (isCheckpointPresent) {
+      sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(cp_.checkpointDir), true)
+      cp_.checkpointDir
+    } else {
+      null
+    }
+  }
+
+  protected[streaming] var checkpointDuration: Duration = if (isCheckpointPresent) cp_.checkpointDuration else null
+  protected[streaming] var receiverJobThread: Thread = null
+  protected[streaming] var scheduler: Scheduler = null
+
+  /**
+   * Return the associated Spark context
+   */
+  def sparkContext = sc
+
+  /**
+   * Set each DStreams in this context to remember RDDs it generated in the last given duration.
+   * DStreams remember RDDs only for a limited duration of time and releases them for garbage
+   * collection. This method allows the developer to specify how to long to remember the RDDs (
+   * if the developer wishes to query old data outside the DStream computation).
+   * @param duration Minimum duration that each DStream should remember its RDDs
+   */
+  def remember(duration: Duration) {
+    graph.remember(duration)
+  }
+
+  /**
+   * Set the context to periodically checkpoint the DStream operations for master
+   * fault-tolerance. The graph will be checkpointed every batch interval.
+   * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored
+   */
+  def checkpoint(directory: String) {
+    if (directory != null) {
+      sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
+      checkpointDir = directory
+    } else {
+      checkpointDir = null
+    }
+  }
+
+  protected[streaming] def initialCheckpoint: Checkpoint = {
+    if (isCheckpointPresent) cp_ else null
+  }
+
+  protected[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement()
+
+  /**
+   * Create an input stream with any arbitrary user implemented network receiver.
+   * @param receiver Custom implementation of NetworkReceiver
+   */
+  def networkStream[T: ClassManifest](
+    receiver: NetworkReceiver[T]): DStream[T] = {
+    val inputStream = new PluggableInputDStream[T](this,
+      receiver)
+    graph.addInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create an input stream with any arbitrary user implemented actor receiver.
+   * @param props Props object defining creation of the actor
+   * @param name Name of the actor
+   * @param storageLevel RDD storage level. Defaults to memory-only.
+   *
+   * @note An important point to note:
+   *       Since Actor may exist outside the spark framework, It is thus user's responsibility
+   *       to ensure the type safety, i.e parametrized type of data received and actorStream
+   *       should be same.
+   */
+  def actorStream[T: ClassManifest](
+    props: Props,
+    name: String,
+    storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
+    supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy): DStream[T] = {
+    networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy))
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel RDD storage level. Defaults to memory-only.
+   */
+  def zeroMQStream[T: ClassManifest](
+      publisherUrl:String,
+      subscribe: Subscribe,
+      bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T],
+      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
+      supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
+    ): DStream[T] = {
+    actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)),
+        "ZeroMQReceiver", storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+   * By default the value is pulled from zookeper.
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+   */
+  def kafkaStream[T: ClassManifest](
+      zkQuorum: String,
+      groupId: String,
+      topics: Map[String, Int],
+      initialOffsets: Map[KafkaPartitionKey, Long] = Map[KafkaPartitionKey, Long](),
+      storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
+    ): DStream[T] = {
+    val inputStream = new KafkaInputDStream[T](this, zkQuorum, groupId, topics, initialOffsets, storageLevel)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream from TCP source hostname:port. Data is received using
+   * a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited
+   * lines.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+   */
+  def socketTextStream(
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[String] = {
+    socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
+  }
+
+  /**
+   * Create a input stream from TCP source hostname:port. Data is received using
+   * a TCP socket and the receive bytes it interepreted as object using the given
+   * converter.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @param converter     Function to convert the byte stream to objects
+   * @param storageLevel  Storage level to use for storing the received objects
+   * @tparam T            Type of the objects received (after converting bytes to objects)
+   */
+  def socketStream[T: ClassManifest](
+      hostname: String,
+      port: Int,
+      converter: (InputStream) => Iterator[T],
+      storageLevel: StorageLevel
+    ): DStream[T] = {
+    val inputStream = new SocketInputDStream[T](this, hostname, port, converter, storageLevel)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def flumeStream (
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[SparkFlumeEvent] = {
+    val inputStream = new FlumeInputDStream(this, hostname, port, storageLevel)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream from network source hostname:port, where data is received
+   * as serialized blocks (serialized using the Spark's serializer) that can be directly
+   * pushed into the block manager without deserializing them. This is the most efficient
+   * way to receive data.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @param storageLevel  Storage level to use for storing the received objects
+   * @tparam T            Type of the objects in the received blocks
+   */
+  def rawSocketStream[T: ClassManifest](
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[T] = {
+    val inputStream = new RawInputDStream[T](this, hostname, port, storageLevel)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream that monitors a Hadoop-compatible filesystem
+   * for new files and reads them using the given key-value types and input format.
+   * File names starting with . are ignored.
+   * @param directory HDFS directory to monitor for new file
+   * @tparam K Key type for reading HDFS file
+   * @tparam V Value type for reading HDFS file
+   * @tparam F Input format for reading HDFS file
+   */
+  def fileStream[
+    K: ClassManifest,
+    V: ClassManifest,
+    F <: NewInputFormat[K, V]: ClassManifest
+  ] (directory: String): DStream[(K, V)] = {
+    val inputStream = new FileInputDStream[K, V, F](this, directory)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream that monitors a Hadoop-compatible filesystem
+   * for new files and reads them using the given key-value types and input format.
+   * @param directory HDFS directory to monitor for new file
+   * @param filter Function to filter paths to process
+   * @param newFilesOnly Should process only new files and ignore existing files in the directory
+   * @tparam K Key type for reading HDFS file
+   * @tparam V Value type for reading HDFS file
+   * @tparam F Input format for reading HDFS file
+   */
+  def fileStream[
+    K: ClassManifest,
+    V: ClassManifest,
+    F <: NewInputFormat[K, V]: ClassManifest
+  ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = {
+    val inputStream = new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a input stream that monitors a Hadoop-compatible filesystem
+   * for new files and reads them as text files (using key as LongWritable, value
+   * as Text and input format as TextInputFormat). File names starting with . are ignored.
+   * @param directory HDFS directory to monitor for new file
+   */
+  def textFileStream(directory: String): DStream[String] = {
+    fileStream[LongWritable, Text, TextInputFormat](directory).map(_._2.toString)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param username Twitter username
+   * @param password Twitter password
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(
+      username: String,
+      password: String,
+      filters: Seq[String] = Nil,
+      storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+    ): DStream[Status] = {
+    val inputStream = new TwitterInputDStream(this, username, password, filters, storageLevel)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create an input stream from a queue of RDDs. In each batch,
+   * it will process either one or all of the RDDs returned by the queue.
+   * @param queue      Queue of RDDs
+   * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+   * @tparam T         Type of objects in the RDD
+   */
+  def queueStream[T: ClassManifest](
+      queue: Queue[RDD[T]],
+      oneAtATime: Boolean = true
+    ): DStream[T] = {
+    queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1))
+  }
+
+  /**
+   * Create an input stream from a queue of RDDs. In each batch,
+   * it will process either one or all of the RDDs returned by the queue.
+   * @param queue      Queue of RDDs
+   * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+   * @param defaultRDD Default RDD is returned by the DStream when the queue is empty. Set as null if no RDD should be returned when empty
+   * @tparam T         Type of objects in the RDD
+   */
+  def queueStream[T: ClassManifest](
+      queue: Queue[RDD[T]],
+      oneAtATime: Boolean,
+      defaultRDD: RDD[T]
+    ): DStream[T] = {
+    val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
+    registerInputStream(inputStream)
+    inputStream
+  }
+
+  /**
+   * Create a unified DStream from multiple DStreams of the same type and same interval
+   */
+  def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = {
+    new UnionDStream[T](streams.toArray)
+  }
+
+  /**
+   * Register an input stream that will be started (InputDStream.start() called) to get the
+   * input data.
+   */
+  def registerInputStream(inputStream: InputDStream[_]) {
+    graph.addInputStream(inputStream)
+  }
+
+  /**
+   * Register an output stream that will be computed every interval
+   */
+  def registerOutputStream(outputStream: DStream[_]) {
+    graph.addOutputStream(outputStream)
+  }
+
+  protected def validate() {
+    assert(graph != null, "Graph is null")
+    graph.validate()
+
+    assert(
+      checkpointDir == null || checkpointDuration != null,
+      "Checkpoint directory has been set, but the graph checkpointing interval has " +
+        "not been set. Please use StreamingContext.checkpoint() to set the interval."
+    )
+  }
+
+  /**
+   * Start the execution of the streams.
+   */
+  def start() {
+    if (checkpointDir != null && checkpointDuration == null && graph != null) {
+      checkpointDuration = graph.batchDuration
+    }
+
+    validate()
+
+    val networkInputStreams = graph.getInputStreams().filter(s => s match {
+        case n: NetworkInputDStream[_] => true
+        case _ => false
+      }).map(_.asInstanceOf[NetworkInputDStream[_]]).toArray
+
+    if (networkInputStreams.length > 0) {
+      // Start the network input tracker (must start before receivers)
+      networkInputTracker = new NetworkInputTracker(this, networkInputStreams)
+      networkInputTracker.start()
+    }
+
+    Thread.sleep(1000)
+
+    // Start the scheduler
+    scheduler = new Scheduler(this)
+    scheduler.start()
+  }
+
+  /**
+   * Stop the execution of the streams.
+   */
+  def stop() {
+    try {
+      if (scheduler != null) scheduler.stop()
+      if (networkInputTracker != null) networkInputTracker.stop()
+      if (receiverJobThread != null) receiverJobThread.interrupt()
+      sc.stop()
+      logInfo("StreamingContext stopped successfully")
+    } catch {
+      case e: Exception => logWarning("Error while stopping", e)
+    }
+  }
+}
+
+
+object StreamingContext {
+
+  implicit def toPairDStreamFunctions[K: ClassManifest, V: ClassManifest](stream: DStream[(K,V)]) = {
+    new PairDStreamFunctions[K, V](stream)
+  }
+
+  protected[streaming] def createNewSparkContext(
+      master: String,
+      appName: String,
+      sparkHome: String,
+      jars: Seq[String],
+      environment: Map[String, String]): SparkContext = {
+    // Set the default cleaner delay to an hour if not already set.
+    // This should be sufficient for even 1 second interval.
+    if (MetadataCleaner.getDelaySeconds < 0) {
+      MetadataCleaner.setDelaySeconds(3600)
+    }
+    new SparkContext(master, appName, sparkHome, jars, environment)
+  }
+
+  protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
+    if (prefix == null) {
+      time.milliseconds.toString
+    } else if (suffix == null || suffix.length ==0) {
+      prefix + "-" + time.milliseconds
+    } else {
+      prefix + "-" + time.milliseconds + "." + suffix
+    }
+  }
+
+  protected[streaming] def getSparkCheckpointDir(sscCheckpointDir: String): String = {
+    new Path(sscCheckpointDir, UUID.randomUUID.toString).toString
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/Time.scala b/streaming/src/main/scala/spark/streaming/Time.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f14decf08ba8f98bc4f59e062b0b8ea17d7ade1e
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/Time.scala
@@ -0,0 +1,55 @@
+package spark.streaming
+
+/**
+ * This is a simple class that represents an absolute instant of time.
+ * Internally, it represents time as the difference, measured in milliseconds, between the current
+ * time and midnight, January 1, 1970 UTC. This is the same format as what is returned by
+ * System.currentTimeMillis.
+ */
+case class Time(private val millis: Long) {
+
+  def milliseconds: Long = millis
+
+  def < (that: Time): Boolean = (this.millis < that.millis)
+
+  def <= (that: Time): Boolean = (this.millis <= that.millis)
+
+  def > (that: Time): Boolean = (this.millis > that.millis)
+  
+  def >= (that: Time): Boolean = (this.millis >= that.millis)
+
+  def + (that: Duration): Time = new Time(millis + that.milliseconds)
+
+  def - (that: Time): Duration = new Duration(millis - that.millis)
+
+  def - (that: Duration): Time = new Time(millis - that.milliseconds)
+
+  def floor(that: Duration): Time = {
+    val t = that.milliseconds
+    val m = math.floor(this.millis / t).toLong 
+    new Time(m * t)
+  }
+
+  def isMultipleOf(that: Duration): Boolean =
+    (this.millis % that.milliseconds == 0)
+
+  def min(that: Time): Time = if (this < that) this else that
+
+  def max(that: Time): Time = if (this > that) this else that
+
+  def until(that: Time, interval: Duration): Seq[Time] = {
+    (this.milliseconds) until (that.milliseconds) by (interval.milliseconds) map (new Time(_))
+  }
+
+  def to(that: Time, interval: Duration): Seq[Time] = {
+    (this.milliseconds) to (that.milliseconds) by (interval.milliseconds) map (new Time(_))
+  }
+
+
+  override def toString: String = (millis.toString + " ms")
+
+}
+
+object Time {
+  val ordering = Ordering.by((time: Time) => time.millis)
+}
\ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..4d93f0a5f729e48cbda0c18f6104449ccb48abc7
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
@@ -0,0 +1,85 @@
+package spark.streaming.api.java
+
+import spark.streaming.{Duration, Time, DStream}
+import spark.api.java.function.{Function => JFunction}
+import spark.api.java.JavaRDD
+import spark.storage.StorageLevel
+import spark.RDD
+
+/**
+ * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[spark.RDD]]
+ * for more details on RDDs). DStreams can either be created from live data (such as, data from
+ * HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
+ * such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
+ * DStream periodically generates a RDD, either from live data or by transforming the RDD generated
+ * by a parent DStream.
+ *
+ * This class contains the basic operations available on all DStreams, such as `map`, `filter` and
+ * `window`. In addition, [[spark.streaming.api.java.JavaPairDStream]] contains operations available
+ * only on DStreams of key-value pairs, such as `groupByKeyAndWindow` and `join`.
+ *
+ * DStreams internally is characterized by a few basic properties:
+ *  - A list of other DStreams that the DStream depends on
+ *  - A time interval at which the DStream generates an RDD
+ *  - A function that is used to generate an RDD after each time interval
+ */
+class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassManifest[T])
+    extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] {
+
+  override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd)
+
+  /** Return a new DStream containing only the elements that satisfy a predicate. */
+  def filter(f: JFunction[T, java.lang.Boolean]): JavaDStream[T] =
+    dstream.filter((x => f(x).booleanValue()))
+
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  def cache(): JavaDStream[T] = dstream.cache()
+
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  def persist(): JavaDStream[T] = dstream.persist()
+
+  /** Persist the RDDs of this DStream with the given storage level */
+  def persist(storageLevel: StorageLevel): JavaDStream[T] = dstream.persist(storageLevel)
+
+  /** Generate an RDD for the given duration */
+  def compute(validTime: Time): JavaRDD[T] = {
+    dstream.compute(validTime) match {
+      case Some(rdd) => new JavaRDD(rdd)
+      case None => null
+    }
+  }
+
+  /**
+   * Return a new DStream in which each RDD contains all the elements in seen in a
+   * sliding window of time over this DStream. The new DStream generates RDDs with
+   * the same interval as this DStream.
+   * @param windowDuration width of the window; must be a multiple of this DStream's interval.
+   */
+  def window(windowDuration: Duration): JavaDStream[T] =
+    dstream.window(windowDuration)
+
+  /**
+   * Return a new DStream in which each RDD contains all the elements in seen in a
+   * sliding window of time over this DStream.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def window(windowDuration: Duration, slideDuration: Duration): JavaDStream[T] =
+    dstream.window(windowDuration, slideDuration)
+
+  /**
+   * Return a new DStream by unifying data of another DStream with this DStream.
+   * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream.
+   */
+  def union(that: JavaDStream[T]): JavaDStream[T] =
+    dstream.union(that.dstream)
+}
+
+object JavaDStream {
+  implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] =
+    new JavaDStream[T](dstream)
+}
\ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
new file mode 100644
index 0000000000000000000000000000000000000000..548809a359644d21785bc4f8c96951a42f1c07f3
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
@@ -0,0 +1,299 @@
+package spark.streaming.api.java
+
+import java.util.{List => JList}
+import java.lang.{Long => JLong}
+
+import scala.collection.JavaConversions._
+
+import spark.streaming._
+import spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD}
+import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
+import java.util
+import spark.RDD
+import JavaDStream._
+
+trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]
+    extends Serializable {
+  implicit val classManifest: ClassManifest[T]
+
+  def dstream: DStream[T]
+
+  def wrapRDD(in: RDD[T]): R
+
+  implicit def scalaIntToJavaLong(in: DStream[Long]): JavaDStream[JLong] = {
+    in.map(new JLong(_))
+  }
+
+  /**
+   * Print the first ten elements of each RDD generated in this DStream. This is an output
+   * operator, so this DStream will be registered as an output stream and there materialized.
+   */
+  def print() = dstream.print()
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by counting each RDD
+   * of this DStream.
+   */
+  def count(): JavaDStream[JLong] = dstream.count()
+
+  /**
+   * Return a new DStream in which each RDD contains the counts of each distinct value in
+   * each RDD of this DStream.  Hash partitioning is used to generate the RDDs with
+   * Spark's default number of partitions.
+   */
+  def countByValue(): JavaPairDStream[T, JLong] = {
+    JavaPairDStream.scalaToJavaLong(dstream.countByValue())
+  }
+
+  /**
+   * Return a new DStream in which each RDD contains the counts of each distinct value in
+   * each RDD of this DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
+   * partitions.
+   * @param numPartitions  number of partitions of each RDD in the new DStream.
+   */
+  def countByValue(numPartitions: Int): JavaPairDStream[T, JLong] = {
+    JavaPairDStream.scalaToJavaLong(dstream.countByValue(numPartitions))
+  }
+
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by counting the number
+   * of elements in a window over this DStream. windowDuration and slideDuration are as defined in the
+   * window() operation. This is equivalent to window(windowDuration, slideDuration).count()
+   */
+  def countByWindow(windowDuration: Duration, slideDuration: Duration) : JavaDStream[JLong] = {
+    dstream.countByWindow(windowDuration, slideDuration)
+  }
+
+  /**
+   * Return a new DStream in which each RDD contains the count of distinct elements in
+   * RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with
+   * Spark's default number of partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration)
+    : JavaPairDStream[T, JLong] = {
+    JavaPairDStream.scalaToJavaLong(
+      dstream.countByValueAndWindow(windowDuration, slideDuration))
+  }
+
+  /**
+   * Return a new DStream in which each RDD contains the count of distinct elements in
+   * RDDs in a sliding window over this DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
+   * partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  number of partitions of each RDD in the new DStream.
+   */
+  def countByValueAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int)
+    : JavaPairDStream[T, JLong] = {
+    JavaPairDStream.scalaToJavaLong(
+      dstream.countByValueAndWindow(windowDuration, slideDuration, numPartitions))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying glom() to each RDD of
+   * this DStream. Applying glom() to an RDD coalesces all elements within each partition into
+   * an array.
+   */
+  def glom(): JavaDStream[JList[T]] =
+    new JavaDStream(dstream.glom().map(x => new java.util.ArrayList[T](x.toSeq)))
+
+  /** Return the StreamingContext associated with this DStream */
+  def context(): StreamingContext = dstream.context()
+
+  /** Return a new DStream by applying a function to all elements of this DStream. */
+  def map[R](f: JFunction[T, R]): JavaDStream[R] = {
+    new JavaDStream(dstream.map(f)(f.returnType()))(f.returnType())
+  }
+
+  /** Return a new DStream by applying a function to all elements of this DStream. */
+  def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = {
+    def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+    new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType())
+  }
+
+  /**
+   * Return a new DStream by applying a function to all elements of this DStream,
+   * and then flattening the results
+   */
+  def flatMap[U](f: FlatMapFunction[T, U]): JavaDStream[U] = {
+    import scala.collection.JavaConverters._
+    def fn = (x: T) => f.apply(x).asScala
+    new JavaDStream(dstream.flatMap(fn)(f.elementType()))(f.elementType())
+  }
+
+  /**
+   * Return a new DStream by applying a function to all elements of this DStream,
+   * and then flattening the results
+   */
+  def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = {
+    import scala.collection.JavaConverters._
+    def fn = (x: T) => f.apply(x).asScala
+    def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+    new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType())
+  }
+
+    /**
+   * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
+   * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
+   * of the RDD.
+   */
+  def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = {
+    def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
+    new JavaDStream(dstream.mapPartitions(fn)(f.elementType()))(f.elementType())
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs
+   * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
+   * of the RDD.
+   */
+  def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2])
+  : JavaPairDStream[K2, V2] = {
+    def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
+    new JavaPairDStream(dstream.mapPartitions(fn))(f.keyType(), f.valueType())
+  }
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by reducing each RDD
+   * of this DStream.
+   */
+  def reduce(f: JFunction2[T, T, T]): JavaDStream[T] = dstream.reduce(f)
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by reducing all
+   * elements in a sliding window over this DStream.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByWindow(
+      reduceFunc: (T, T) => T,
+      windowDuration: Duration,
+      slideDuration: Duration
+    ): DStream[T] = {
+    dstream.reduceByWindow(reduceFunc, windowDuration, slideDuration)
+  }
+
+
+  /**
+   * Return a new DStream in which each RDD has a single element generated by reducing all
+   * elements in a sliding window over this DStream. However, the reduction is done incrementally
+   * using the old window's reduced value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   *  This is more efficient than reduceByWindow without "inverse reduce" function.
+   *  However, it is applicable to only "invertible reduce functions".
+   * @param reduceFunc associative reduce function
+   * @param invReduceFunc inverse reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByWindow(
+      reduceFunc: JFunction2[T, T, T],
+      invReduceFunc: JFunction2[T, T, T],
+      windowDuration: Duration,
+      slideDuration: Duration
+    ): JavaDStream[T] = {
+    dstream.reduceByWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration)
+  }
+
+  /**
+   * Return all the RDDs between 'fromDuration' to 'toDuration' (both included)
+   */
+  def slice(fromTime: Time, toTime: Time): JList[R] = {
+    new util.ArrayList(dstream.slice(fromTime, toTime).map(wrapRDD(_)).toSeq)
+  }
+
+  /**
+   * Apply a function to each RDD in this DStream. This is an output operator, so
+   * this DStream will be registered as an output stream and therefore materialized.
+   */
+  def foreach(foreachFunc: JFunction[R, Void]) {
+    dstream.foreach(rdd => foreachFunc.call(wrapRDD(rdd)))
+  }
+
+  /**
+   * Apply a function to each RDD in this DStream. This is an output operator, so
+   * this DStream will be registered as an output stream and therefore materialized.
+   */
+  def foreach(foreachFunc: JFunction2[R, Time, Void]) {
+    dstream.foreach((rdd, time) => foreachFunc.call(wrapRDD(rdd), time))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of this DStream.
+   */
+  def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = {
+    implicit val cm: ClassManifest[U] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+    def scalaTransform (in: RDD[T]): RDD[U] =
+      transformFunc.call(wrapRDD(in)).rdd
+    dstream.transform(scalaTransform(_))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of this DStream.
+   */
+  def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = {
+    implicit val cm: ClassManifest[U] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+    def scalaTransform (in: RDD[T], time: Time): RDD[U] =
+      transformFunc.call(wrapRDD(in), time).rdd
+    dstream.transform(scalaTransform(_, _))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of this DStream.
+   */
+  def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]):
+  JavaPairDStream[K2, V2] = {
+    implicit val cmk: ClassManifest[K2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
+    implicit val cmv: ClassManifest[V2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+    def scalaTransform (in: RDD[T]): RDD[(K2, V2)] =
+      transformFunc.call(wrapRDD(in)).rdd
+    dstream.transform(scalaTransform(_))
+  }
+
+  /**
+   * Return a new DStream in which each RDD is generated by applying a function
+   * on each RDD of this DStream.
+   */
+  def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]):
+  JavaPairDStream[K2, V2] = {
+    implicit val cmk: ClassManifest[K2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
+    implicit val cmv: ClassManifest[V2] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+    def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] =
+      transformFunc.call(wrapRDD(in), time).rdd
+    dstream.transform(scalaTransform(_, _))
+  }
+
+  /**
+   * Enable periodic checkpointing of RDDs of this DStream
+   * @param interval Time interval after which generated RDD will be checkpointed
+   */
+  def checkpoint(interval: Duration) = {
+    dstream.checkpoint(interval)
+  }
+}
\ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..30240cad988be32e82c701a0dd7b535156c3dcb4
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
@@ -0,0 +1,599 @@
+package spark.streaming.api.java
+
+import java.util.{List => JList}
+import java.lang.{Long => JLong}
+
+import scala.collection.JavaConversions._
+
+import spark.streaming._
+import spark.streaming.StreamingContext._
+import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import spark.{RDD, Partitioner}
+import org.apache.hadoop.mapred.{JobConf, OutputFormat}
+import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
+import org.apache.hadoop.conf.Configuration
+import spark.api.java.{JavaRDD, JavaPairRDD}
+import spark.storage.StorageLevel
+import com.google.common.base.Optional
+import spark.RDD
+
+class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
+    implicit val kManifiest: ClassManifest[K],
+    implicit val vManifest: ClassManifest[V])
+    extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] {
+
+  override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
+
+  // =======================================================================
+  // Methods common to all DStream's
+  // =======================================================================
+
+  /** Return a new DStream containing only the elements that satisfy a predicate. */
+  def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairDStream[K, V] =
+    dstream.filter((x => f(x).booleanValue()))
+
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  def cache(): JavaPairDStream[K, V] = dstream.cache()
+
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  def persist(): JavaPairDStream[K, V] = dstream.persist()
+
+  /** Persist the RDDs of this DStream with the given storage level */
+  def persist(storageLevel: StorageLevel): JavaPairDStream[K, V] = dstream.persist(storageLevel)
+
+  /** Method that generates a RDD for the given Duration */
+  def compute(validTime: Time): JavaPairRDD[K, V] = {
+    dstream.compute(validTime) match {
+      case Some(rdd) => new JavaPairRDD(rdd)
+      case None => null
+    }
+  }
+
+  /**
+   * Return a new DStream which is computed based on windowed batches of this DStream.
+   * The new DStream generates RDDs with the same interval as this DStream.
+   * @param windowDuration width of the window; must be a multiple of this DStream's interval.
+   * @return
+   */
+  def window(windowDuration: Duration): JavaPairDStream[K, V] =
+    dstream.window(windowDuration)
+
+  /**
+   * Return a new DStream which is computed based on windowed batches of this DStream.
+   * @param windowDuration duration (i.e., width) of the window;
+   *                   must be a multiple of this DStream's interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                   the new DStream will generate RDDs); must be a multiple of this
+   *                   DStream's interval
+   */
+  def window(windowDuration: Duration, slideDuration: Duration): JavaPairDStream[K, V] =
+    dstream.window(windowDuration, slideDuration)
+
+  /**
+   * Return a new DStream by unifying data of another DStream with this DStream.
+   * @param that Another DStream having the same interval (i.e., slideDuration) as this DStream.
+   */
+  def union(that: JavaPairDStream[K, V]): JavaPairDStream[K, V] =
+    dstream.union(that.dstream)
+
+  // =======================================================================
+  // Methods only for PairDStream's
+  // =======================================================================
+
+  /**
+   * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * generate the RDDs with Spark's default number of partitions.
+   */
+  def groupByKey(): JavaPairDStream[K, JList[V]] =
+    dstream.groupByKey().mapValues(seqAsJavaList _)
+
+  /**
+   * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * generate the RDDs with `numPartitions` partitions.
+   */
+  def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] =
+    dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _)
+
+  /**
+   * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream.
+   * Therefore, the values for each key in `this` DStream's RDDs are grouped into a
+   * single sequence to generate the RDDs of the new DStream. [[spark.Partitioner]]
+   * is used to control the partitioning of each RDD.
+   */
+  def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] =
+    dstream.groupByKey(partitioner).mapValues(seqAsJavaList _)
+
+  /**
+   * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * merged using the associative reduce function. Hash partitioning is used to generate the RDDs
+   * with Spark's default number of partitions.
+   */
+  def reduceByKey(func: JFunction2[V, V, V]): JavaPairDStream[K, V] =
+    dstream.reduceByKey(func)
+
+  /**
+   * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * merged using the supplied reduce function. Hash partitioning is used to generate the RDDs
+   * with `numPartitions` partitions.
+   */
+  def reduceByKey(func: JFunction2[V, V, V], numPartitions: Int): JavaPairDStream[K, V] =
+    dstream.reduceByKey(func, numPartitions)
+
+  /**
+   * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * merged using the supplied reduce function. [[spark.Partitioner]] is used to control the
+   * partitioning of each RDD.
+   */
+  def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = {
+    dstream.reduceByKey(func, partitioner)
+  }
+
+  /**
+   * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+   * combineByKey for RDDs. Please refer to combineByKey in [[spark.PairRDDFunctions]] for more
+   * information.
+   */
+  def combineByKey[C](createCombiner: JFunction[V, C],
+      mergeValue: JFunction2[C, V, C],
+      mergeCombiners: JFunction2[C, C, C],
+      partitioner: Partitioner
+    ): JavaPairDStream[K, C] = {
+    implicit val cm: ClassManifest[C] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+    dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner)
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window. This is similar to
+   * `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
+   * with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
+   * Spark's default number of partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   */
+  def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JList[V]] = {
+    dstream.groupByKeyAndWindow(windowDuration).mapValues(seqAsJavaList _)
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window. Similar to
+   * `DStream.groupByKey()`, but applies it over a sliding window. Hash partitioning is used to
+   * generate the RDDs with Spark's default number of partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration)
+  : JavaPairDStream[K, JList[V]] = {
+    dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(seqAsJavaList _)
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  Number of partitions of each RDD in the new DStream.
+   */
+  def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int)
+  :JavaPairDStream[K, JList[V]] = {
+    dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions)
+      .mapValues(seqAsJavaList _)
+  }
+
+  /**
+   * Return a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * Similar to `DStream.groupByKey()`, but applies it over a sliding window.
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+   */
+  def groupByKeyAndWindow(
+      windowDuration: Duration,
+      slideDuration: Duration,
+      partitioner: Partitioner
+    ):JavaPairDStream[K, JList[V]] = {
+    dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner)
+      .mapValues(seqAsJavaList _)
+  }
+
+  /**
+   * Create a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+   * Similar to `DStream.reduceByKey()`, but applies it over a sliding window. The new DStream
+   * generates RDDs with the same interval as this DStream. Hash partitioning is used to generate
+   * the RDDs with Spark's default number of partitions.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   */
+  def reduceByKeyAndWindow(reduceFunc: Function2[V, V, V], windowDuration: Duration)
+  :JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(reduceFunc, windowDuration)
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+   * generate the RDDs with Spark's default number of partitions.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: Function2[V, V, V],
+      windowDuration: Duration,
+      slideDuration: Duration
+    ):JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration)
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * `DStream.reduceByKey()` but applies it over a sliding window. Hash partitioning is used to
+   * generate the RDDs with `numPartitions` partitions.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  Number of partitions of each RDD in the new DStream.
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: Function2[V, V, V],
+      windowDuration: Duration,
+      slideDuration: Duration,
+      numPartitions: Int
+    ): JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, numPartitions)
+  }
+
+  /**
+   * Return a new DStream by applying `reduceByKey` over a sliding window. Similar to
+   * `DStream.reduceByKey()`, but applies it over a sliding window.
+   * @param reduceFunc associative reduce function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: Function2[V, V, V],
+      windowDuration: Duration,
+      slideDuration: Duration,
+      partitioner: Partitioner
+    ): JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(reduceFunc, windowDuration, slideDuration, partitioner)
+  }
+
+  /**
+   * Return a new DStream by reducing over a using incremental computation.
+   * The reduced value of over a new window is calculated using the old window's reduce value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
+   * However, it is applicable to only "invertible reduce functions".
+   * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+   * @param reduceFunc associative reduce function
+   * @param invReduceFunc inverse function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: Function2[V, V, V],
+      invReduceFunc: Function2[V, V, V],
+      windowDuration: Duration,
+      slideDuration: Duration
+    ): JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration)
+  }
+
+  /**
+   * Return a new DStream by applying incremental `reduceByKey` over a sliding window.
+   * The reduced value of over a new window is calculated using the old window's reduce value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
+   * However, it is applicable to only "invertible reduce functions".
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   * @param reduceFunc associative reduce function
+   * @param invReduceFunc inverse function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param numPartitions  number of partitions of each RDD in the new DStream.
+   * @param filterFunc     function to filter expired key-value pairs;
+   *                       only pairs that satisfy the function are retained
+   *                       set this to null if you do not want to filter
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: Function2[V, V, V],
+      invReduceFunc: Function2[V, V, V],
+      windowDuration: Duration,
+      slideDuration: Duration,
+      numPartitions: Int,
+      filterFunc: JFunction[(K, V), java.lang.Boolean]
+    ): JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(
+        reduceFunc,
+        invReduceFunc,
+        windowDuration,
+        slideDuration,
+        numPartitions,
+        (p: (K, V)) => filterFunc(p).booleanValue()
+    )
+  }
+
+  /**
+   * Return a new DStream by applying incremental `reduceByKey` over a sliding window.
+   * The reduced value of over a new window is calculated using the old window's reduce value :
+   *  1. reduce the new values that entered the window (e.g., adding new counts)
+   *  2. "inverse reduce" the old values that left the window (e.g., subtracting old counts)
+   * This is more efficient that reduceByKeyAndWindow without "inverse reduce" function.
+   * However, it is applicable to only "invertible reduce functions".
+   * @param reduceFunc associative reduce function
+   * @param invReduceFunc inverse function
+   * @param windowDuration width of the window; must be a multiple of this DStream's
+   *                       batching interval
+   * @param slideDuration  sliding interval of the window (i.e., the interval after which
+   *                       the new DStream will generate RDDs); must be a multiple of this
+   *                       DStream's batching interval
+   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+   * @param filterFunc     function to filter expired key-value pairs;
+   *                       only pairs that satisfy the function are retained
+   *                       set this to null if you do not want to filter
+   */
+  def reduceByKeyAndWindow(
+      reduceFunc: Function2[V, V, V],
+      invReduceFunc: Function2[V, V, V],
+      windowDuration: Duration,
+      slideDuration: Duration,
+      partitioner: Partitioner,
+      filterFunc: JFunction[(K, V), java.lang.Boolean]
+  ): JavaPairDStream[K, V] = {
+    dstream.reduceByKeyAndWindow(
+        reduceFunc,
+        invReduceFunc,
+        windowDuration,
+        slideDuration,
+        partitioner,
+        (p: (K, V)) => filterFunc(p).booleanValue()
+    )
+  }
+
+  private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]):
+  (Seq[V], Option[S]) => Option[S] = {
+    val scalaFunc: (Seq[V], Option[S]) => Option[S] = (values, state) => {
+      val list: JList[V] = values
+      val scalaState: Optional[S] = state match {
+        case Some(s) => Optional.of(s)
+        case _ => Optional.absent()
+      }
+      val result: Optional[S] = in.apply(list, scalaState)
+      result.isPresent match {
+        case true => Some(result.get())
+        case _ => None
+      }
+    }
+    scalaFunc
+  }
+
+  /**
+   * Create a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of each key.
+   * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated.
+   * @tparam S State type
+   */
+  def updateStateByKey[S](updateFunc: JFunction2[JList[V], Optional[S], Optional[S]])
+  : JavaPairDStream[K, S] = {
+    implicit val cm: ClassManifest[S] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
+    dstream.updateStateByKey(convertUpdateStateFunction(updateFunc))
+  }
+
+  /**
+   * Create a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of each key.
+   * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated.
+   * @param numPartitions Number of partitions of each RDD in the new DStream.
+   * @tparam S State type
+   */
+  def updateStateByKey[S: ClassManifest](
+      updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
+      numPartitions: Int)
+  : JavaPairDStream[K, S] = {
+    dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), numPartitions)
+  }
+
+  /**
+   * Create a new "state" DStream where the state for each key is updated by applying
+   * the given function on the previous state of the key and the new values of the key.
+   * [[spark.Partitioner]] is used to control the partitioning of each RDD.
+   * @param updateFunc State update function. If `this` function returns None, then
+   *                   corresponding state key-value pair will be eliminated.
+   * @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
+   * @tparam S State type
+   */
+  def updateStateByKey[S: ClassManifest](
+      updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
+      partitioner: Partitioner
+  ): JavaPairDStream[K, S] = {
+    dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner)
+  }
+
+  def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = {
+    implicit val cm: ClassManifest[U] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+    dstream.mapValues(f)
+  }
+
+  def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = {
+    import scala.collection.JavaConverters._
+    def fn = (x: V) => f.apply(x).asScala
+    implicit val cm: ClassManifest[U] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+    dstream.flatMapValues(fn)
+  }
+
+  /**
+   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+   * key in both RDDs. HashPartitioner is used to partition each generated RDD into default number
+   * of partitions.
+   */
+  def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
+  }
+
+  /**
+   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+   * or `other` DStreams, the generated RDD will contains a tuple with the list of values for that
+   * key in both RDDs. Partitioner is used to partition each generated RDD.
+   */
+  def cogroup[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
+  : JavaPairDStream[K, (JList[V], JList[W])] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    dstream.cogroup(other.dstream, partitioner)
+        .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
+  }
+
+  /**
+   * Join `this` DStream with `other` DStream. HashPartitioner is used
+   * to partition each generated RDD into default number of partitions.
+   */
+  def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    dstream.join(other.dstream)
+  }
+
+  /**
+   * Join `this` DStream with `other` DStream, that is, each RDD of the new DStream will
+   * be generated by joining RDDs from `this` and other DStream. Uses the given
+   * Partitioner to partition each generated RDD.
+   */
+  def join[W](other: JavaPairDStream[K, W], partitioner: Partitioner)
+  : JavaPairDStream[K, (V, W)] = {
+    implicit val cm: ClassManifest[W] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+    dstream.join(other.dstream, partitioner)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsHadoopFiles[F <: OutputFormat[K, V]](prefix: String, suffix: String) {
+    dstream.saveAsHadoopFiles(prefix, suffix)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsHadoopFiles(
+      prefix: String,
+      suffix: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: OutputFormat[_, _]]) {
+    dstream.saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsHadoopFiles(
+      prefix: String,
+      suffix: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: OutputFormat[_, _]],
+      conf: JobConf) {
+    dstream.saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](prefix: String, suffix: String) {
+    dstream.saveAsNewAPIHadoopFiles(prefix, suffix)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsNewAPIHadoopFiles(
+      prefix: String,
+      suffix: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: NewOutputFormat[_, _]]) {
+    dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass)
+  }
+
+  /**
+   * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is
+   * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix".
+   */
+  def saveAsNewAPIHadoopFiles(
+      prefix: String,
+      suffix: String,
+      keyClass: Class[_],
+      valueClass: Class[_],
+      outputFormatClass: Class[_ <: NewOutputFormat[_, _]],
+      conf: Configuration = new Configuration) {
+    dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf)
+  }
+
+  override val classManifest: ClassManifest[(K, V)] =
+    implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+}
+
+object JavaPairDStream {
+  implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)])
+  :JavaPairDStream[K, V] =
+    new JavaPairDStream[K, V](dstream)
+
+  def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = {
+    implicit val cmk: ClassManifest[K] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+    implicit val cmv: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+    new JavaPairDStream[K, V](dstream.dstream)
+  }
+
+  def scalaToJavaLong[K: ClassManifest](dstream: JavaPairDStream[K, Long])
+  : JavaPairDStream[K, JLong] = {
+    StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_))
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3d149a742cb3cf37de71c26d8e6aa77bdb7de408
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -0,0 +1,575 @@
+package spark.streaming.api.java
+
+import spark.streaming._
+import receivers.{ActorReceiver, ReceiverSupervisorStrategy}
+import spark.streaming.dstream._
+import spark.storage.StorageLevel
+
+import spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
+import spark.api.java.{JavaSparkContext, JavaRDD}
+
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+
+import twitter4j.Status
+
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
+import akka.zeromq.Subscribe
+
+import scala.collection.JavaConversions._
+
+import java.lang.{Long => JLong, Integer => JInt}
+import java.io.InputStream
+import java.util.{Map => JMap}
+
+/**
+ * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
+ * information (such as, cluster URL and job name) to internally create a SparkContext, it provides
+ * methods used to create DStream from various input sources.
+ */
+class JavaStreamingContext(val ssc: StreamingContext) {
+
+  // TODOs:
+  // - Test to/from Hadoop functions
+  // - Support creating and registering InputStreams
+
+
+  /**
+   * Creates a StreamingContext.
+   * @param master Name of the Spark Master
+   * @param appName Name to be used when registering with the scheduler
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(master: String, appName: String, batchDuration: Duration) =
+    this(new StreamingContext(master, appName, batchDuration, null, Nil, Map()))
+
+  /**
+   * Creates a StreamingContext.
+   * @param master Name of the Spark Master
+   * @param appName Name to be used when registering with the scheduler
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param sparkHome The SPARK_HOME directory on the slave nodes
+   * @param jarFile JAR file containing job code, to ship to cluster. This can be a path on the local
+   *                file system or an HDFS, HTTP, HTTPS, or FTP URL.
+   */
+  def this(
+      master: String,
+      appName: String,
+      batchDuration: Duration,
+      sparkHome: String,
+      jarFile: String) =
+    this(new StreamingContext(master, appName, batchDuration, sparkHome, Seq(jarFile), Map()))
+
+  /**
+   * Creates a StreamingContext.
+   * @param master Name of the Spark Master
+   * @param appName Name to be used when registering with the scheduler
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param sparkHome The SPARK_HOME directory on the slave nodes
+   * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
+   *             system or HDFS, HTTP, HTTPS, or FTP URLs.
+   */
+  def this(
+      master: String,
+      appName: String,
+      batchDuration: Duration,
+      sparkHome: String,
+      jars: Array[String]) =
+    this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, Map()))
+
+  /**
+   * Creates a StreamingContext.
+   * @param master Name of the Spark Master
+   * @param appName Name to be used when registering with the scheduler
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   * @param sparkHome The SPARK_HOME directory on the slave nodes
+   * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
+   *             system or HDFS, HTTP, HTTPS, or FTP URLs.
+   * @param environment Environment variables to set on worker nodes
+   */
+  def this(
+    master: String,
+    appName: String,
+    batchDuration: Duration,
+    sparkHome: String,
+    jars: Array[String],
+    environment: JMap[String, String]) =
+    this(new StreamingContext(master, appName, batchDuration, sparkHome, jars, environment))
+
+  /**
+   * Creates a StreamingContext using an existing SparkContext.
+   * @param sparkContext The underlying JavaSparkContext to use
+   * @param batchDuration The time interval at which streaming data will be divided into batches
+   */
+  def this(sparkContext: JavaSparkContext, batchDuration: Duration) =
+    this(new StreamingContext(sparkContext.sc, batchDuration))
+
+  /**
+   * Re-creates a StreamingContext from a checkpoint file.
+   * @param path Path either to the directory that was specified as the checkpoint directory, or
+   *             to the checkpoint file 'graph' or 'graph.bk'.
+   */
+  def this(path: String) = this (new StreamingContext(path))
+
+  /** The underlying SparkContext */
+  val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   */
+  def kafkaStream[T](
+    zkQuorum: String,
+    groupId: String,
+    topics: JMap[String, JInt])
+  : JavaDStream[T] = {
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+   * By default the value is pulled from zookeper.
+   */
+  def kafkaStream[T](
+    zkQuorum: String,
+    groupId: String,
+    topics: JMap[String, JInt],
+    initialOffsets: JMap[KafkaPartitionKey, JLong])
+  : JavaDStream[T] = {
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.kafkaStream[T](
+      zkQuorum,
+      groupId,
+      Map(topics.mapValues(_.intValue()).toSeq: _*),
+      Map(initialOffsets.mapValues(_.longValue()).toSeq: _*))
+  }
+
+  /**
+   * Create an input stream that pulls messages form a Kafka Broker.
+   * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+   * @param groupId The group id for this consumer.
+   * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+   * in its own thread.
+   * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+   * By default the value is pulled from zookeper.
+   * @param storageLevel RDD storage level. Defaults to memory-only
+   */
+  def kafkaStream[T](
+    zkQuorum: String,
+    groupId: String,
+    topics: JMap[String, JInt],
+    initialOffsets: JMap[KafkaPartitionKey, JLong],
+    storageLevel: StorageLevel)
+  : JavaDStream[T] = {
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.kafkaStream[T](
+      zkQuorum,
+      groupId,
+      Map(topics.mapValues(_.intValue()).toSeq: _*),
+      Map(initialOffsets.mapValues(_.longValue()).toSeq: _*),
+      storageLevel)
+  }
+
+  /**
+   * Create a input stream from network source hostname:port. Data is received using
+   * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
+   * lines.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+   */
+  def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel)
+  : JavaDStream[String] = {
+    ssc.socketTextStream(hostname, port, storageLevel)
+  }
+
+  /**
+   * Create a input stream from network source hostname:port. Data is received using
+   * a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
+   * lines.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   */
+  def socketTextStream(hostname: String, port: Int): JavaDStream[String] = {
+    ssc.socketTextStream(hostname, port)
+  }
+
+  /**
+   * Create a input stream from network source hostname:port. Data is received using
+   * a TCP socket and the receive bytes it interepreted as object using the given
+   * converter.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @param converter     Function to convert the byte stream to objects
+   * @param storageLevel  Storage level to use for storing the received objects
+   * @tparam T            Type of the objects received (after converting bytes to objects)
+   */
+  def socketStream[T](
+      hostname: String,
+      port: Int,
+      converter: JFunction[InputStream, java.lang.Iterable[T]],
+      storageLevel: StorageLevel)
+  : JavaDStream[T] = {
+    def fn = (x: InputStream) => converter.apply(x).toIterator
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.socketStream(hostname, port, fn, storageLevel)
+  }
+
+  /**
+   * Creates a input stream that monitors a Hadoop-compatible filesystem
+   * for new files and reads them as text files (using key as LongWritable, value
+   * as Text and input format as TextInputFormat). File names starting with . are ignored.
+   * @param directory HDFS directory to monitor for new file
+   */
+  def textFileStream(directory: String): JavaDStream[String] = {
+    ssc.textFileStream(directory)
+  }
+
+  /**
+   * Create a input stream from network source hostname:port, where data is received
+   * as serialized blocks (serialized using the Spark's serializer) that can be directly
+   * pushed into the block manager without deserializing them. This is the most efficient
+   * way to receive data.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @param storageLevel  Storage level to use for storing the received objects
+   * @tparam T            Type of the objects in the received blocks
+   */
+  def rawSocketStream[T](
+      hostname: String,
+      port: Int,
+      storageLevel: StorageLevel): JavaDStream[T] = {
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel))
+  }
+
+  /**
+   * Create a input stream from network source hostname:port, where data is received
+   * as serialized blocks (serialized using the Spark's serializer) that can be directly
+   * pushed into the block manager without deserializing them. This is the most efficient
+   * way to receive data.
+   * @param hostname      Hostname to connect to for receiving data
+   * @param port          Port to connect to for receiving data
+   * @tparam T            Type of the objects in the received blocks
+   */
+  def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = {
+    implicit val cmt: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port))
+  }
+
+  /**
+   * Creates a input stream that monitors a Hadoop-compatible filesystem
+   * for new files and reads them using the given key-value types and input format.
+   * File names starting with . are ignored.
+   * @param directory HDFS directory to monitor for new file
+   * @tparam K Key type for reading HDFS file
+   * @tparam V Value type for reading HDFS file
+   * @tparam F Input format for reading HDFS file
+   */
+  def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = {
+    implicit val cmk: ClassManifest[K] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+    implicit val cmv: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+    implicit val cmf: ClassManifest[F] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]]
+    ssc.fileStream[K, V, F](directory);
+  }
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
+    JavaDStream[SparkFlumeEvent] = {
+    ssc.flumeStream(hostname, port, storageLevel)
+  }
+
+
+  /**
+   * Creates a input stream from a Flume source.
+   * @param hostname Hostname of the slave machine to which the flume data will be sent
+   * @param port     Port of the slave machine to which the flume data will be sent
+   */
+  def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
+    ssc.flumeStream(hostname, port)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param username Twitter username
+   * @param password Twitter password
+   * @param filters Set of filter strings to get only those tweets that match them
+   * @param storageLevel Storage level to use for storing the received objects
+   */
+  def twitterStream(
+      username: String,
+      password: String,
+      filters: Array[String],
+      storageLevel: StorageLevel
+    ): JavaDStream[Status] = {
+    ssc.twitterStream(username, password, filters, storageLevel)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param username Twitter username
+   * @param password Twitter password
+   * @param filters Set of filter strings to get only those tweets that match them
+   */
+  def twitterStream(
+      username: String,
+      password: String,
+      filters: Array[String]
+    ): JavaDStream[Status] = {
+    ssc.twitterStream(username, password, filters)
+  }
+
+  /**
+   * Create a input stream that returns tweets received from Twitter.
+   * @param username Twitter username
+   * @param password Twitter password
+   */
+  def twitterStream(
+      username: String,
+      password: String
+    ): JavaDStream[Status] = {
+    ssc.twitterStream(username, password)
+  }
+
+  /**
+   * Create an input stream with any arbitrary user implemented actor receiver.
+   * @param props Props object defining creation of the actor
+   * @param name Name of the actor
+   * @param storageLevel Storage level to use for storing the received objects
+   *
+   * @note An important point to note:
+   *       Since Actor may exist outside the spark framework, It is thus user's responsibility
+   *       to ensure the type safety, i.e parametrized type of data received and actorStream
+   *       should be same.
+   */
+  def actorStream[T](
+      props: Props,
+      name: String,
+      storageLevel: StorageLevel,
+      supervisorStrategy: SupervisorStrategy
+    ): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.actorStream[T](props, name, storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream with any arbitrary user implemented actor receiver.
+   * @param props Props object defining creation of the actor
+   * @param name Name of the actor
+   * @param storageLevel Storage level to use for storing the received objects
+   *
+   * @note An important point to note:
+   *       Since Actor may exist outside the spark framework, It is thus user's responsibility
+   *       to ensure the type safety, i.e parametrized type of data received and actorStream
+   *       should be same.
+   */
+  def actorStream[T](
+      props: Props,
+      name: String,
+      storageLevel: StorageLevel
+  ): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.actorStream[T](props, name, storageLevel)
+  }
+
+  /**
+   * Create an input stream with any arbitrary user implemented actor receiver.
+   * @param props Props object defining creation of the actor
+   * @param name Name of the actor
+   *
+   * @note An important point to note:
+   *       Since Actor may exist outside the spark framework, It is thus user's responsibility
+   *       to ensure the type safety, i.e parametrized type of data received and actorStream
+   *       should be same.
+   */
+  def actorStream[T](
+      props: Props,
+      name: String
+    ): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.actorStream[T](props, name)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel  Storage level to use for storing the received objects
+   */
+  def zeroMQStream[T](
+      publisherUrl:String,
+      subscribe: Subscribe,
+      bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T],
+      storageLevel: StorageLevel,
+      supervisorStrategy: SupervisorStrategy
+    ): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   * @param storageLevel RDD storage level. Defaults to memory-only.
+   */
+  def zeroMQStream[T](
+      publisherUrl:String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+      storageLevel: StorageLevel
+    ): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
+  }
+
+  /**
+   * Create an input stream that receives messages pushed by a zeromq publisher.
+   * @param publisherUrl Url of remote zeromq publisher
+   * @param subscribe topic to subscribe to
+   * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+   *                       of byte thus it needs the converter(which might be deserializer of bytes)
+   *                       to translate from sequence of sequence of bytes, where sequence refer to a frame
+   *                       and sub sequence refer to its payload.
+   */
+  def zeroMQStream[T](
+      publisherUrl:String,
+      subscribe: Subscribe,
+      bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
+    ): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+    ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
+  }
+
+  /**
+   * Registers an output stream that will be computed every interval
+   */
+  def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) {
+    ssc.registerOutputStream(outputStream.dstream)
+  }
+
+  /**
+   * Creates a input stream from an queue of RDDs. In each batch,
+   * it will process either one or all of the RDDs returned by the queue.
+   *
+   * NOTE: changes to the queue after the stream is created will not be recognized.
+   * @param queue      Queue of RDDs
+   * @tparam T         Type of objects in the RDD
+   */
+  def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+    sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
+    ssc.queueStream(sQueue)
+  }
+
+  /**
+   * Creates a input stream from an queue of RDDs. In each batch,
+   * it will process either one or all of the RDDs returned by the queue.
+   *
+   * NOTE: changes to the queue after the stream is created will not be recognized.
+   * @param queue      Queue of RDDs
+   * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+   * @tparam T         Type of objects in the RDD
+   */
+  def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+    sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
+    ssc.queueStream(sQueue, oneAtATime)
+  }
+
+  /**
+   * Creates a input stream from an queue of RDDs. In each batch,
+   * it will process either one or all of the RDDs returned by the queue.
+   *
+   * NOTE: changes to the queue after the stream is created will not be recognized.
+   * @param queue      Queue of RDDs
+   * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
+   * @param defaultRDD Default RDD is returned by the DStream when the queue is empty
+   * @tparam T         Type of objects in the RDD
+   */
+  def queueStream[T](
+      queue: java.util.Queue[JavaRDD[T]],
+      oneAtATime: Boolean,
+      defaultRDD: JavaRDD[T]): JavaDStream[T] = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    val sQueue = new scala.collection.mutable.Queue[spark.RDD[T]]
+    sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
+    ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd)
+  }
+
+  /**
+   * Sets the context to periodically checkpoint the DStream operations for master
+   * fault-tolerance. The graph will be checkpointed every batch interval.
+   * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored
+   */
+  def checkpoint(directory: String) {
+    ssc.checkpoint(directory)
+  }
+
+  /**
+   * Sets each DStreams in this context to remember RDDs it generated in the last given duration.
+   * DStreams remember RDDs only for a limited duration of duration and releases them for garbage
+   * collection. This method allows the developer to specify how to long to remember the RDDs (
+   * if the developer wishes to query old data outside the DStream computation).
+   * @param duration Minimum duration that each DStream should remember its RDDs
+   */
+  def remember(duration: Duration) {
+    ssc.remember(duration)
+  }
+
+  /**
+   * Starts the execution of the streams.
+   */
+  def start() = ssc.start()
+
+  /**
+   * Sstops the execution of the streams.
+   */
+  def stop() = ssc.stop()
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..4ef4bb7de1023f2a3159cccb58d70aeb56a20bf5
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/CoGroupedDStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.dstream
+
+import spark.{RDD, Partitioner}
+import spark.rdd.CoGroupedRDD
+import spark.streaming.{Time, DStream, Duration}
+
+private[streaming]
+class CoGroupedDStream[K : ClassManifest](
+    parents: Seq[DStream[(K, _)]],
+    partitioner: Partitioner
+  ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) {
+
+  if (parents.length == 0) {
+    throw new IllegalArgumentException("Empty array of parents")
+  }
+
+  if (parents.map(_.ssc).distinct.size > 1) {
+    throw new IllegalArgumentException("Array of parents have different StreamingContexts")
+  }
+
+  if (parents.map(_.slideDuration).distinct.size > 1) {
+    throw new IllegalArgumentException("Array of parents have different slide times")
+  }
+
+  override def dependencies = parents.toList
+
+  override def slideDuration: Duration = parents.head.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = {
+    val part = partitioner
+    val rdds = parents.flatMap(_.getOrCompute(validTime))
+    if (rdds.size > 0) {
+      val q = new CoGroupedRDD[K](rdds, part)
+      Some(q)
+    } else {
+      None
+    }
+  }
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..41c3af46948e67975be1efd2cc249c4d50f7700c
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ConstantInputDStream.scala
@@ -0,0 +1,19 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.streaming.{Time, StreamingContext}
+
+/**
+ * An input stream that always returns the same RDD on each timestep. Useful for testing.
+ */
+class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T])
+  extends InputDStream[T](ssc_) {
+
+  override def start() {}
+
+  override def stop() {}
+
+  override def compute(validTime: Time): Option[RDD[T]] = {
+    Some(rdd)
+  }
+}
\ No newline at end of file
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..41b9bd9461d9cc5ec93efe43bebdad62de0d9223
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
@@ -0,0 +1,182 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.rdd.UnionRDD
+import spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
+
+import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+
+import scala.collection.mutable.{HashSet, HashMap}
+import java.io.{ObjectInputStream, IOException}
+
+private[streaming]
+class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest](
+    @transient ssc_ : StreamingContext,
+    directory: String,
+    filter: Path => Boolean = FileInputDStream.defaultFilter,
+    newFilesOnly: Boolean = true) 
+  extends InputDStream[(K, V)](ssc_) {
+
+  protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
+
+  // Latest file mod time seen till any point of time
+  private val lastModTimeFiles = new HashSet[String]()
+  private var lastModTime = 0L
+
+  @transient private var path_ : Path = null
+  @transient private var fs_ : FileSystem = null
+  @transient private[streaming] var files = new HashMap[Time, Array[String]]
+
+  override def start() {
+    if (newFilesOnly) {
+      lastModTime = graph.zeroTime.milliseconds
+    } else {
+      lastModTime = 0
+    }
+    logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly)
+  }
+  
+  override def stop() { }
+
+  /**
+   * Finds the files that were modified since the last time this method was called and makes
+   * a union RDD out of them. Note that this maintains the list of files that were processed
+   * in the latest modification time in the previous call to this method. This is because the
+   * modification time returned by the FileStatus API seems to return times only at the
+   * granularity of seconds. And new files may have the same modification time as the
+   * latest modification time in the previous call to this method yet was not reported in
+   * the previous call.
+   */
+  override def compute(validTime: Time): Option[RDD[(K, V)]] = {
+    assert(validTime.milliseconds >= lastModTime, "Trying to get new files for really old time [" + validTime + " < " + lastModTime)
+
+    // Create the filter for selecting new files
+    val newFilter = new PathFilter() {
+      // Latest file mod time seen in this round of fetching files and its corresponding files
+      var latestModTime = 0L
+      val latestModTimeFiles = new HashSet[String]()
+
+      def accept(path: Path): Boolean = {
+        if (!filter(path)) {  // Reject file if it does not satisfy filter
+          logDebug("Rejected by filter " + path)
+          return false
+        } else {              // Accept file only if
+          val modTime = fs.getFileStatus(path).getModificationTime()
+          logDebug("Mod time for " + path + " is " + modTime)
+          if (modTime < lastModTime) {
+            logDebug("Mod time less than last mod time")
+            return false  // If the file was created before the last time it was called
+          } else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) {
+            logDebug("Mod time equal to last mod time, but file considered already")
+            return false  // If the file was created exactly as lastModTime but not reported yet
+          } else if (modTime > validTime.milliseconds) {
+            logDebug("Mod time more than valid time")
+            return false  // If the file was created after the time this function call requires
+          }
+          if (modTime > latestModTime) {
+            latestModTime = modTime
+            latestModTimeFiles.clear()
+            logDebug("Latest mod time updated to " + latestModTime)
+          }
+          latestModTimeFiles += path.toString
+          logDebug("Accepted " + path)
+          return true
+        }        
+      }
+    }
+    logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime)
+    val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString)
+    logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n"))
+    if (newFiles.length > 0) {
+      // Update the modification time and the files processed for that modification time
+      if (lastModTime != newFilter.latestModTime) {
+        lastModTime = newFilter.latestModTime
+        lastModTimeFiles.clear()
+      }
+      lastModTimeFiles ++= newFilter.latestModTimeFiles
+      logDebug("Last mod time updated to " + lastModTime)
+    }
+    files += ((validTime, newFiles))
+    Some(filesToRDD(newFiles))
+  }
+
+  /** Clear the old time-to-files mappings along with old RDDs */
+  protected[streaming] override def clearOldMetadata(time: Time) {
+    super.clearOldMetadata(time)
+    val oldFiles = files.filter(_._1 <= (time - rememberDuration))
+    files --= oldFiles.keys
+    logInfo("Cleared " + oldFiles.size + " old files that were older than " +
+      (time - rememberDuration) + ": " + oldFiles.keys.mkString(", "))
+    logDebug("Cleared files are:\n" +
+      oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n"))
+  }
+
+  /** Generate one RDD from an array of files */
+  protected[streaming] def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
+    new UnionRDD(
+      context.sparkContext,
+      files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
+    )
+  }
+
+  private def path: Path = {
+    if (path_ == null) path_ = new Path(directory)
+    path_
+  }
+
+  private def fs: FileSystem = {
+    if (fs_ == null) fs_ = path.getFileSystem(new Configuration())
+    fs_
+  }
+
+  @throws(classOf[IOException])
+  private def readObject(ois: ObjectInputStream) {
+    logDebug(this.getClass().getSimpleName + ".readObject used")
+    ois.defaultReadObject()
+    generatedRDDs = new HashMap[Time, RDD[(K,V)]] ()
+    files = new HashMap[Time, Array[String]]
+  }
+
+  /**
+   * A custom version of the DStreamCheckpointData that stores names of
+   * Hadoop files as checkpoint data.
+   */
+  private[streaming]
+  class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) {
+
+    def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]]
+
+    override def update() {
+      hadoopFiles.clear()
+      hadoopFiles ++= files
+    }
+
+    override def cleanup() { }
+
+    override def restore() {
+      hadoopFiles.foreach {
+        case (t, f) => {
+          // Restore the metadata in both files and generatedRDDs
+          logInfo("Restoring files for time " + t + " - " +
+            f.mkString("[", ", ", "]") )
+          files += ((t, f))
+          generatedRDDs += ((t, filesToRDD(f)))
+        }
+      }
+    }
+
+    override def toString() = {
+      "[\n" + hadoopFiles.size + " file sets\n" +
+        hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]"
+    }
+  }
+}
+
+private[streaming]
+object FileInputDStream {
+  def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".")
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e993164f9949231fd0e583c0df5a7902afda7e34
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FilteredDStream.scala
@@ -0,0 +1,21 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class FilteredDStream[T: ClassManifest](
+    parent: DStream[T],
+    filterFunc: T => Boolean
+  ) extends DStream[T](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[T]] = {
+    parent.getOrCompute(validTime).map(_.filter(filterFunc))
+  }
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..cabd34f5f2e763a0a3c32c77ff85836d64cf100a
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMapValuedDStream.scala
@@ -0,0 +1,20 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+import spark.SparkContext._
+
+private[streaming]
+class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+    parent: DStream[(K, V)],
+    flatMapValueFunc: V => TraversableOnce[U]
+  ) extends DStream[(K, U)](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[(K, U)]] = {
+    parent.getOrCompute(validTime).map(_.flatMapValues[U](flatMapValueFunc))
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a69af6058903b850ffbb32319ece644fbd8d7de5
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlatMappedDStream.scala
@@ -0,0 +1,20 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class FlatMappedDStream[T: ClassManifest, U: ClassManifest](
+    parent: DStream[T],
+    flatMapFunc: T => Traversable[U]
+  ) extends DStream[U](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[U]] = {
+    parent.getOrCompute(validTime).map(_.flatMap(flatMapFunc))
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c9644b3a83aa053e96519cab70d5754b6b6803c3
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
@@ -0,0 +1,137 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext
+
+import spark.Utils
+import spark.storage.StorageLevel
+
+import org.apache.flume.source.avro.AvroSourceProtocol
+import org.apache.flume.source.avro.AvroFlumeEvent
+import org.apache.flume.source.avro.Status
+import org.apache.avro.ipc.specific.SpecificResponder
+import org.apache.avro.ipc.NettyServer
+
+import scala.collection.JavaConversions._
+
+import java.net.InetSocketAddress
+import java.io.{ObjectInput, ObjectOutput, Externalizable}
+import java.nio.ByteBuffer
+
+private[streaming]
+class FlumeInputDStream[T: ClassManifest](
+  @transient ssc_ : StreamingContext,
+  host: String,
+  port: Int,
+  storageLevel: StorageLevel
+) extends NetworkInputDStream[SparkFlumeEvent](ssc_) {
+
+  override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = {
+    new FlumeReceiver(host, port, storageLevel)
+  }
+}
+
+/**
+ * A wrapper class for AvroFlumeEvent's with a custom serialization format.
+ *
+ * This is necessary because AvroFlumeEvent uses inner data structures
+ * which are not serializable.
+ */
+class SparkFlumeEvent() extends Externalizable {
+  var event : AvroFlumeEvent = new AvroFlumeEvent()
+
+  /* De-serialize from bytes. */
+  def readExternal(in: ObjectInput) {
+    val bodyLength = in.readInt()
+    val bodyBuff = new Array[Byte](bodyLength)
+    in.read(bodyBuff)
+
+    val numHeaders = in.readInt()
+    val headers = new java.util.HashMap[CharSequence, CharSequence]
+
+    for (i <- 0 until numHeaders) {
+      val keyLength = in.readInt()
+      val keyBuff = new Array[Byte](keyLength)
+      in.read(keyBuff)
+      val key : String = Utils.deserialize(keyBuff)
+
+      val valLength = in.readInt()
+      val valBuff = new Array[Byte](valLength)
+      in.read(valBuff)
+      val value : String = Utils.deserialize(valBuff)
+
+      headers.put(key, value)
+    }
+
+    event.setBody(ByteBuffer.wrap(bodyBuff))
+    event.setHeaders(headers)
+  }
+
+  /* Serialize to bytes. */
+  def writeExternal(out: ObjectOutput) {
+    val body = event.getBody.array()
+    out.writeInt(body.length)
+    out.write(body)
+
+    val numHeaders = event.getHeaders.size()
+    out.writeInt(numHeaders)
+    for ((k, v) <- event.getHeaders) {
+      val keyBuff = Utils.serialize(k.toString)
+      out.writeInt(keyBuff.length)
+      out.write(keyBuff)
+      val valBuff = Utils.serialize(v.toString)
+      out.writeInt(valBuff.length)
+      out.write(valBuff)
+    }
+  }
+}
+
+private[streaming] object SparkFlumeEvent {
+  def fromAvroFlumeEvent(in : AvroFlumeEvent) : SparkFlumeEvent = {
+    val event = new SparkFlumeEvent
+    event.event = in
+    event
+  }
+}
+
+/** A simple server that implements Flume's Avro protocol. */
+private[streaming]
+class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol {
+  override def append(event : AvroFlumeEvent) : Status = {
+    receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)
+    Status.OK
+  }
+
+  override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = {
+    events.foreach (event =>
+      receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event))
+    Status.OK
+  }
+}
+
+/** A NetworkReceiver which listens for events using the
+  * Flume Avro interface.*/
+private[streaming]
+class FlumeReceiver(
+    host: String,
+    port: Int,
+    storageLevel: StorageLevel
+  ) extends NetworkReceiver[SparkFlumeEvent] {
+
+  lazy val blockGenerator = new BlockGenerator(storageLevel)
+
+  protected override def onStart() {
+    val responder = new SpecificResponder(
+      classOf[AvroSourceProtocol], new FlumeEventServer(this));
+    val server = new NettyServer(responder, new InetSocketAddress(host, port));
+    blockGenerator.start()
+    server.start()
+    logInfo("Flume receiver started")
+  }
+
+  protected override def onStop() {
+    blockGenerator.stop()
+    logInfo("Flume receiver stopped")
+  }
+
+  override def getLocationPreference = Some(host)
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ee69ea5177921e8b172427b6e6f579f18f70a7a3
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ForEachDStream.scala
@@ -0,0 +1,28 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.streaming.{Duration, DStream, Job, Time}
+
+private[streaming]
+class ForEachDStream[T: ClassManifest] (
+    parent: DStream[T],
+    foreachFunc: (RDD[T], Time) => Unit
+  ) extends DStream[Unit](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[Unit]] = None
+
+  override def generateJob(time: Time): Option[Job] = {
+    parent.getOrCompute(time) match {
+      case Some(rdd) =>
+        val jobFunc = () => {
+          foreachFunc(rdd, time)
+        }
+        Some(new Job(time, jobFunc))
+      case None => None
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b589cbd4d50b79ca03959f1de71b9d69f00d5c56
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/GlommedDStream.scala
@@ -0,0 +1,17 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class GlommedDStream[T: ClassManifest](parent: DStream[T])
+  extends DStream[Array[T]](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[Array[T]]] = {
+    parent.getOrCompute(validTime).map(_.glom())
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3c5d43a60955a38273aabd224073c8fabf2830d4
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
@@ -0,0 +1,53 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Time, Duration, StreamingContext, DStream}
+
+/**
+ * This is the abstract base class for all input streams. This class provides to methods
+ * start() and stop() which called by the scheduler to start and stop receiving data/
+ * Input streams that can generated RDDs from new data just by running a service on
+ * the driver node (that is, without running a receiver onworker nodes) can be
+ * implemented by directly subclassing this InputDStream. For example,
+ * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory for
+ * new files and generates RDDs on the new files. For implementing input streams
+ * that requires running a receiver on the worker nodes, use NetworkInputDStream
+ * as the parent class.
+ */
+abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext)
+  extends DStream[T](ssc_) {
+
+  var lastValidTime: Time = null
+
+  /**
+   * Checks whether the 'time' is valid wrt slideDuration for generating RDD.
+   * Additionally it also ensures valid times are in strictly increasing order.
+   * This ensures that InputDStream.compute() is called strictly on increasing
+   * times.
+   */
+  override protected def isTimeValid(time: Time): Boolean = {
+    if (!super.isTimeValid(time)) {
+      false // Time not valid
+    } else {
+      // Time is valid, but check it it is more than lastValidTime
+      if (lastValidTime != null && time < lastValidTime) {
+        logWarning("isTimeValid called with " + time + " where as last valid time is " + lastValidTime)
+      }
+      lastValidTime = time
+      true
+    }
+  }
+
+  override def dependencies = List()
+
+  override def slideDuration: Duration = {
+    if (ssc == null) throw new Exception("ssc is null")
+    if (ssc.graph.batchDuration == null) throw new Exception("batchDuration is null")
+    ssc.graph.batchDuration
+  }
+
+  /** Method called to start receiving data. Subclasses must implement this method. */
+  def start()
+
+  /** Method called to stop receiving data. Subclasses must implement this method. */
+  def stop()
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ddd9becf325e02e649d54d20278e2db37ed795af
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
@@ -0,0 +1,126 @@
+package spark.streaming.dstream
+
+import spark.Logging
+import spark.storage.StorageLevel
+import spark.streaming.{Time, DStreamCheckpointData, StreamingContext}
+
+import java.util.Properties
+import java.util.concurrent.Executors
+
+import kafka.consumer._
+import kafka.message.{Message, MessageSet, MessageAndMetadata}
+import kafka.serializer.StringDecoder
+import kafka.utils.{Utils, ZKGroupTopicDirs}
+import kafka.utils.ZkUtils._
+
+import scala.collection.Map
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+
+
+// Key for a specific Kafka Partition: (broker, topic, group, part)
+case class KafkaPartitionKey(brokerId: Int, topic: String, groupId: String, partId: Int)
+
+/**
+ * Input stream that pulls messages from a Kafka Broker.
+ * 
+ * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param initialOffsets Optional initial offsets for each of the partitions to consume.
+ * By default the value is pulled from zookeper.
+ * @param storageLevel RDD storage level.
+ */
+private[streaming]
+class KafkaInputDStream[T: ClassManifest](
+    @transient ssc_ : StreamingContext,
+    zkQuorum: String,
+    groupId: String,
+    topics: Map[String, Int],
+    initialOffsets: Map[KafkaPartitionKey, Long],
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[T](ssc_ ) with Logging {
+
+
+  def getReceiver(): NetworkReceiver[T] = {
+    new KafkaReceiver(zkQuorum,  groupId, topics, initialOffsets, storageLevel)
+        .asInstanceOf[NetworkReceiver[T]]
+  }
+}
+
+private[streaming]
+class KafkaReceiver(zkQuorum: String, groupId: String,
+  topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], 
+  storageLevel: StorageLevel) extends NetworkReceiver[Any] {
+
+  // Timeout for establishing a connection to Zookeper in ms.
+  val ZK_TIMEOUT = 10000
+
+  // Handles pushing data into the BlockManager
+  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+  // Connection to Kafka
+  var consumerConnector : ZookeeperConsumerConnector = null
+
+  def onStop() {
+    blockGenerator.stop()
+  }
+
+  def onStart() {
+
+    blockGenerator.start()
+
+    // In case we are using multiple Threads to handle Kafka Messages
+    val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _))
+
+    logInfo("Starting Kafka Consumer Stream with group: " + groupId)
+    logInfo("Initial offsets: " + initialOffsets.toString)
+
+    // Zookeper connection properties
+    val props = new Properties()
+    props.put("zk.connect", zkQuorum)
+    props.put("zk.connectiontimeout.ms", ZK_TIMEOUT.toString)
+    props.put("groupid", groupId)
+
+    // Create the connection to the cluster
+    logInfo("Connecting to Zookeper: " + zkQuorum)
+    val consumerConfig = new ConsumerConfig(props)
+    consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector]
+    logInfo("Connected to " + zkQuorum)
+
+    // If specified, set the topic offset
+    setOffsets(initialOffsets)
+
+    // Create Threads for each Topic/Message Stream we are listening
+    val topicMessageStreams = consumerConnector.createMessageStreams(topics, new StringDecoder())
+
+    // Start the messages handler for each partition
+    topicMessageStreams.values.foreach { streams =>
+      streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) }
+    }
+
+  }
+
+  // Overwrites the offets in Zookeper.
+  private def setOffsets(offsets: Map[KafkaPartitionKey, Long]) {
+    offsets.foreach { case(key, offset) =>
+      val topicDirs = new ZKGroupTopicDirs(key.groupId, key.topic)
+      val partitionName = key.brokerId + "-" + key.partId
+      updatePersistentPath(consumerConnector.zkClient,
+        topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString)
+    }
+  }
+
+  // Handles Kafka Messages
+  private class MessageHandler(stream: KafkaStream[String]) extends Runnable {
+    def run() {
+      logInfo("Starting MessageHandler.")
+      stream.takeWhile { msgAndMetadata =>
+        blockGenerator += msgAndMetadata.message
+        // Keep on handling messages
+
+        true
+      }
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..848afecfad599fa51daf60b5576c408f5b957d7f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/MapPartitionedDStream.scala
@@ -0,0 +1,21 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class MapPartitionedDStream[T: ClassManifest, U: ClassManifest](
+    parent: DStream[T],
+    mapPartFunc: Iterator[T] => Iterator[U],
+    preservePartitioning: Boolean
+  ) extends DStream[U](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[U]] = {
+    parent.getOrCompute(validTime).map(_.mapPartitions[U](mapPartFunc, preservePartitioning))
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..6055aa6a0597badea5922adcc2e0ec350e69cd72
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/MapValuedDStream.scala
@@ -0,0 +1,21 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+import spark.SparkContext._
+
+private[streaming]
+class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+    parent: DStream[(K, V)],
+    mapValueFunc: V => U
+  ) extends DStream[(K, U)](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[(K, U)]] = {
+    parent.getOrCompute(validTime).map(_.mapValues[U](mapValueFunc))
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..20818a0cab108f3c578c821559bbaa4301170e5a
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/MappedDStream.scala
@@ -0,0 +1,20 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+
+private[streaming]
+class MappedDStream[T: ClassManifest, U: ClassManifest] (
+    parent: DStream[T],
+    mapFunc: T => U
+  ) extends DStream[U](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[U]] = {
+    parent.getOrCompute(validTime).map(_.map[U](mapFunc))
+  }
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..7385474963ebb70bfd7f8f86a16282e341824c80
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
@@ -0,0 +1,261 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Time, StreamingContext, AddBlocks, RegisterReceiver, DeregisterReceiver}
+
+import spark.{Logging, SparkEnv, RDD}
+import spark.rdd.BlockRDD
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+
+import java.nio.ByteBuffer
+
+import akka.actor.{Props, Actor}
+import akka.pattern.ask
+import akka.dispatch.Await
+import akka.util.duration._
+import spark.streaming.util.{RecurringTimer, SystemClock}
+import java.util.concurrent.ArrayBlockingQueue
+
+/**
+ * Abstract class for defining any InputDStream that has to start a receiver on worker
+ * nodes to receive external data. Specific implementations of NetworkInputDStream must
+ * define the getReceiver() function that gets the receiver object of type
+ * [[spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive
+ * data.
+ * @param ssc_ Streaming context that will execute this input stream
+ * @tparam T Class type of the object of this stream
+ */
+abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : StreamingContext)
+  extends InputDStream[T](ssc_) {
+
+  // This is an unique identifier that is used to match the network receiver with the
+  // corresponding network input stream.
+  val id = ssc.getNewNetworkStreamId()
+
+  /**
+   * Gets the receiver object that will be sent to the worker nodes
+   * to receive data. This method needs to defined by any specific implementation
+   * of a NetworkInputDStream.
+   */
+  def getReceiver(): NetworkReceiver[T]
+
+  // Nothing to start or stop as both taken care of by the NetworkInputTracker.
+  def start() {}
+
+  def stop() {}
+
+  override def compute(validTime: Time): Option[RDD[T]] = {
+    // If this is called for any time before the start time of the context,
+    // then this returns an empty RDD. This may happen when recovering from a
+    // master failure
+    if (validTime >= graph.startTime) {
+      val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)
+      Some(new BlockRDD[T](ssc.sc, blockIds))
+    } else {
+      Some(new BlockRDD[T](ssc.sc, Array[String]()))
+    }
+  }
+}
+
+
+private[streaming] sealed trait NetworkReceiverMessage
+private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage
+private[streaming] case class ReportBlock(blockId: String, metadata: Any) extends NetworkReceiverMessage
+private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage
+
+/**
+ * Abstract class of a receiver that can be run on worker nodes to receive external data. See
+ * [[spark.streaming.dstream.NetworkInputDStream]] for an explanation.
+ */
+abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging {
+
+  initLogging()
+
+  lazy protected val env = SparkEnv.get
+
+  lazy protected val actor = env.actorSystem.actorOf(
+    Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId)
+
+  lazy protected val receivingThread = Thread.currentThread()
+
+  protected var streamId: Int = -1
+
+  /**
+   * This method will be called to start receiving data. All your receiver
+   * starting code should be implemented by defining this function.
+   */
+  protected def onStart()
+
+  /** This method will be called to stop receiving data. */
+  protected def onStop()
+
+  /** Conveys a placement preference (hostname) for this receiver. */
+  def getLocationPreference() : Option[String] = None
+
+  /**
+   * Starts the receiver. First is accesses all the lazy members to
+   * materialize them. Then it calls the user-defined onStart() method to start
+   * other threads, etc required to receiver the data.
+   */
+  def start() {
+    try {
+      // Access the lazy vals to materialize them
+      env
+      actor
+      receivingThread
+
+      // Call user-defined onStart()
+      onStart()
+    } catch {
+      case ie: InterruptedException =>
+        logInfo("Receiving thread interrupted")
+        //println("Receiving thread interrupted")
+      case e: Exception =>
+        stopOnError(e)
+    }
+  }
+
+  /**
+   * Stops the receiver. First it interrupts the main receiving thread,
+   * that is, the thread that called receiver.start(). Then it calls the user-defined
+   * onStop() method to stop other threads and/or do cleanup.
+   */
+  def stop() {
+    receivingThread.interrupt()
+    onStop()
+    //TODO: terminate the actor
+  }
+
+  /**
+   * Stops the receiver and reports to exception to the tracker.
+   * This should be called whenever an exception has happened on any thread
+   * of the receiver.
+   */
+  protected def stopOnError(e: Exception) {
+    logError("Error receiving data", e)
+    stop()
+    actor ! ReportError(e.toString)
+  }
+
+
+  /**
+   * Pushes a block (as iterator of values) into the block manager.
+   */
+  def pushBlock(blockId: String, iterator: Iterator[T], metadata: Any, level: StorageLevel) {
+    val buffer = new ArrayBuffer[T] ++ iterator
+    env.blockManager.put(blockId, buffer.asInstanceOf[ArrayBuffer[Any]], level)
+
+    actor ! ReportBlock(blockId, metadata)
+  }
+
+  /**
+   * Pushes a block (as bytes) into the block manager.
+   */
+  def pushBlock(blockId: String, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
+    env.blockManager.putBytes(blockId, bytes, level)
+    actor ! ReportBlock(blockId, metadata)
+  }
+
+  /** A helper actor that communicates with the NetworkInputTracker */
+  private class NetworkReceiverActor extends Actor {
+    logInfo("Attempting to register with tracker")
+    val ip = System.getProperty("spark.driver.host", "localhost")
+    val port = System.getProperty("spark.driver.port", "7077").toInt
+    val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
+    val tracker = env.actorSystem.actorFor(url)
+    val timeout = 5.seconds
+
+    override def preStart() {
+      val future = tracker.ask(RegisterReceiver(streamId, self))(timeout)
+      Await.result(future, timeout)
+    }
+
+    override def receive() = {
+      case ReportBlock(blockId, metadata) =>
+        tracker ! AddBlocks(streamId, Array(blockId), metadata)
+      case ReportError(msg) =>
+        tracker ! DeregisterReceiver(streamId, msg)
+      case StopReceiver(msg) =>
+        stop()
+        tracker ! DeregisterReceiver(streamId, msg)
+    }
+  }
+
+  protected[streaming] def setStreamId(id: Int) {
+    streamId = id
+  }
+
+  /**
+   * Batches objects created by a [[spark.streaming.NetworkReceiver]] and puts them into
+   * appropriately named blocks at regular intervals. This class starts two threads,
+   * one to periodically start a new batch and prepare the previous batch of as a block,
+   * the other to push the blocks into the block manager.
+   */
+  class BlockGenerator(storageLevel: StorageLevel)
+    extends Serializable with Logging {
+
+    case class Block(id: String, iterator: Iterator[T], metadata: Any = null)
+
+    val clock = new SystemClock()
+    val blockInterval = 200L
+    val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer)
+    val blockStorageLevel = storageLevel
+    val blocksForPushing = new ArrayBlockingQueue[Block](1000)
+    val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } }
+
+    var currentBuffer = new ArrayBuffer[T]
+
+    def start() {
+      blockIntervalTimer.start()
+      blockPushingThread.start()
+      logInfo("Data handler started")
+    }
+
+    def stop() {
+      blockIntervalTimer.stop()
+      blockPushingThread.interrupt()
+      logInfo("Data handler stopped")
+    }
+
+    def += (obj: T) {
+      currentBuffer += obj
+    }
+
+    private def createBlock(blockId: String, iterator: Iterator[T]) : Block = {
+      new Block(blockId, iterator)
+    }
+
+    private def updateCurrentBuffer(time: Long) {
+      try {
+        val newBlockBuffer = currentBuffer
+        currentBuffer = new ArrayBuffer[T]
+        if (newBlockBuffer.size > 0) {
+          val blockId = "input-" + NetworkReceiver.this.streamId + "-" + (time - blockInterval)
+          val newBlock = createBlock(blockId, newBlockBuffer.toIterator)
+          blocksForPushing.add(newBlock)
+        }
+      } catch {
+        case ie: InterruptedException =>
+          logInfo("Block interval timer thread interrupted")
+        case e: Exception =>
+          NetworkReceiver.this.stop()
+      }
+    }
+
+    private def keepPushingBlocks() {
+      logInfo("Block pushing thread started")
+      try {
+        while(true) {
+          val block = blocksForPushing.take()
+          NetworkReceiver.this.pushBlock(block.id, block.iterator, block.metadata, storageLevel)
+        }
+      } catch {
+        case ie: InterruptedException =>
+          logInfo("Block pushing thread interrupted")
+        case e: Exception =>
+          NetworkReceiver.this.stop()
+      }
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3c2a81947b96b5b678b76e432cb865c79892c803
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/PluggableInputDStream.scala
@@ -0,0 +1,13 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext
+
+private[streaming]
+class PluggableInputDStream[T: ClassManifest](
+  @transient ssc_ : StreamingContext,
+  receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) {
+
+  def getReceiver(): NetworkReceiver[T] = {
+    receiver
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..da224ad6f718cbb8a29e1e6f976923f1bfacf6d9
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
@@ -0,0 +1,42 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.rdd.UnionRDD
+
+import scala.collection.mutable.Queue
+import scala.collection.mutable.ArrayBuffer
+import spark.streaming.{Time, StreamingContext}
+
+private[streaming]
+class QueueInputDStream[T: ClassManifest](
+    @transient ssc: StreamingContext,
+    val queue: Queue[RDD[T]],
+    oneAtATime: Boolean,
+    defaultRDD: RDD[T]
+  ) extends InputDStream[T](ssc) {
+  
+  override def start() { }
+  
+  override def stop() { }
+  
+  override def compute(validTime: Time): Option[RDD[T]] = {
+    val buffer = new ArrayBuffer[RDD[T]]()
+    if (oneAtATime && queue.size > 0) {
+      buffer += queue.dequeue()
+    } else {
+      buffer ++= queue
+    }
+    if (buffer.size > 0) {
+      if (oneAtATime) {
+        Some(buffer.head)
+      } else {
+        Some(new UnionRDD(ssc.sc, buffer.toSeq))
+      }
+    } else if (defaultRDD != null) {
+      Some(defaultRDD)
+    } else {
+      None
+    }
+  }
+  
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1b2fa567795d4245567a42362a6f45e4f8d96e8b
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
@@ -0,0 +1,91 @@
+package spark.streaming.dstream
+
+import spark.Logging
+import spark.storage.StorageLevel
+import spark.streaming.StreamingContext
+
+import java.net.InetSocketAddress
+import java.nio.ByteBuffer
+import java.nio.channels.{ReadableByteChannel, SocketChannel}
+import java.io.EOFException
+import java.util.concurrent.ArrayBlockingQueue
+
+
+/**
+ * An input stream that reads blocks of serialized objects from a given network address.
+ * The blocks will be inserted directly into the block store. This is the fastest way to get
+ * data into Spark Streaming, though it requires the sender to batch data and serialize it
+ * in the format that the system is configured with.
+ */
+private[streaming]
+class RawInputDStream[T: ClassManifest](
+    @transient ssc_ : StreamingContext,
+    host: String,
+    port: Int,
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[T](ssc_ ) with Logging {
+
+  def getReceiver(): NetworkReceiver[T] = {
+    new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]]
+  }
+}
+
+private[streaming]
+class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel)
+  extends NetworkReceiver[Any] {
+
+  var blockPushingThread: Thread = null
+
+  override def getLocationPreference = None
+
+  def onStart() {
+    // Open a socket to the target address and keep reading from it
+    logInfo("Connecting to " + host + ":" + port)
+    val channel = SocketChannel.open()
+    channel.configureBlocking(true)
+    channel.connect(new InetSocketAddress(host, port))
+    logInfo("Connected to " + host + ":" + port)
+
+    val queue = new ArrayBlockingQueue[ByteBuffer](2)
+
+    blockPushingThread = new Thread {
+      setDaemon(true)
+      override def run() {
+        var nextBlockNumber = 0
+        while (true) {
+          val buffer = queue.take()
+          val blockId = "input-" + streamId + "-" + nextBlockNumber
+          nextBlockNumber += 1
+          pushBlock(blockId, buffer, null, storageLevel)
+        }
+      }
+    }
+    blockPushingThread.start()
+
+    val lengthBuffer = ByteBuffer.allocate(4)
+    while (true) {
+      lengthBuffer.clear()
+      readFully(channel, lengthBuffer)
+      lengthBuffer.flip()
+      val length = lengthBuffer.getInt()
+      val dataBuffer = ByteBuffer.allocate(length)
+      readFully(channel, dataBuffer)
+      dataBuffer.flip()
+      logInfo("Read a block with " + length + " bytes")
+      queue.put(dataBuffer)
+    }
+  }
+
+  def onStop() {
+    if (blockPushingThread != null) blockPushingThread.interrupt()
+  }
+
+  /** Read a buffer fully from a given Channel */
+  private def readFully(channel: ReadableByteChannel, dest: ByteBuffer) {
+    while (dest.position < dest.limit) {
+      if (channel.read(dest) == -1) {
+        throw new EOFException("End of channel")
+      }
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..343b6915e79a36b02292bace746681474f0e3294
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -0,0 +1,157 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext._
+
+import spark.RDD
+import spark.rdd.{CoGroupedRDD, MapPartitionsRDD}
+import spark.Partitioner
+import spark.SparkContext._
+import spark.storage.StorageLevel
+
+import scala.collection.mutable.ArrayBuffer
+import spark.streaming.{Duration, Interval, Time, DStream}
+
+private[streaming]
+class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
+    parent: DStream[(K, V)],
+    reduceFunc: (V, V) => V,
+    invReduceFunc: (V, V) => V,
+    filterFunc: Option[((K, V)) => Boolean],
+    _windowDuration: Duration,
+    _slideDuration: Duration,
+    partitioner: Partitioner
+  ) extends DStream[(K,V)](parent.ssc) {
+
+  assert(_windowDuration.isMultipleOf(parent.slideDuration),
+    "The window duration of ReducedWindowedDStream (" + _slideDuration + ") " +
+      "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")"
+  )
+
+  assert(_slideDuration.isMultipleOf(parent.slideDuration),
+    "The slide duration of ReducedWindowedDStream (" + _slideDuration + ") " +
+      "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")"
+  )
+
+  // Reduce each batch of data using reduceByKey which will be further reduced by window 
+  // by ReducedWindowedDStream
+  val reducedStream = parent.reduceByKey(reduceFunc, partitioner)
+
+  // Persist RDDs to memory by default as these RDDs are going to be reused.
+  super.persist(StorageLevel.MEMORY_ONLY_SER)
+  reducedStream.persist(StorageLevel.MEMORY_ONLY_SER)
+
+  def windowDuration: Duration =  _windowDuration
+
+  override def dependencies = List(reducedStream)
+
+  override def slideDuration: Duration = _slideDuration
+
+  override val mustCheckpoint = true
+
+  override def parentRememberDuration: Duration = rememberDuration + windowDuration
+
+  override def persist(storageLevel: StorageLevel): DStream[(K,V)] = {
+    super.persist(storageLevel)
+    reducedStream.persist(storageLevel)
+    this
+  }
+
+  override def checkpoint(interval: Duration): DStream[(K, V)] = {
+    super.checkpoint(interval)
+    //reducedStream.checkpoint(interval)
+    this
+  }
+
+  override def compute(validTime: Time): Option[RDD[(K, V)]] = {
+    val reduceF = reduceFunc
+    val invReduceF = invReduceFunc
+
+    val currentTime = validTime
+    val currentWindow = new Interval(currentTime - windowDuration + parent.slideDuration, currentTime)
+    val previousWindow = currentWindow - slideDuration
+
+    logDebug("Window time = " + windowDuration)
+    logDebug("Slide time = " + slideDuration)
+    logDebug("ZeroTime = " + zeroTime)
+    logDebug("Current window = " + currentWindow)
+    logDebug("Previous window = " + previousWindow)
+
+    //  _____________________________
+    // |  previous window   _________|___________________
+    // |___________________|       current window        |  --------------> Time
+    //                     |_____________________________|
+    //
+    // |________ _________|          |________ _________|
+    //          |                             |
+    //          V                             V
+    //       old RDDs                     new RDDs
+    //
+
+    // Get the RDDs of the reduced values in "old time steps"
+    val oldRDDs =
+      reducedStream.slice(previousWindow.beginTime, currentWindow.beginTime - parent.slideDuration)
+    logDebug("# old RDDs = " + oldRDDs.size)
+
+    // Get the RDDs of the reduced values in "new time steps"
+    val newRDDs =
+      reducedStream.slice(previousWindow.endTime + parent.slideDuration, currentWindow.endTime)
+    logDebug("# new RDDs = " + newRDDs.size)
+
+    // Get the RDD of the reduced value of the previous window
+    val previousWindowRDD =
+      getOrCompute(previousWindow.endTime).getOrElse(ssc.sc.makeRDD(Seq[(K,V)]()))
+
+    // Make the list of RDDs that needs to cogrouped together for reducing their reduced values
+    val allRDDs = new ArrayBuffer[RDD[(K, V)]]() += previousWindowRDD ++= oldRDDs ++= newRDDs
+
+    // Cogroup the reduced RDDs and merge the reduced values
+    val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(K, _)]]], partitioner)
+    //val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _
+
+    val numOldValues = oldRDDs.size
+    val numNewValues = newRDDs.size
+
+    val mergeValues = (seqOfValues: Seq[Seq[V]]) => {
+      if (seqOfValues.size != 1 + numOldValues + numNewValues) {
+        throw new Exception("Unexpected number of sequences of reduced values")
+      }
+      // Getting reduced values "old time steps" that will be removed from current window
+      val oldValues = (1 to numOldValues).map(i => seqOfValues(i)).filter(!_.isEmpty).map(_.head)
+      // Getting reduced values "new time steps"
+      val newValues =
+        (1 to numNewValues).map(i => seqOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head)
+
+      if (seqOfValues(0).isEmpty) {
+        // If previous window's reduce value does not exist, then at least new values should exist
+        if (newValues.isEmpty) {
+          throw new Exception("Neither previous window has value for key, nor new values found. " +
+            "Are you sure your key class hashes consistently?")
+        }
+        // Reduce the new values
+        newValues.reduce(reduceF) // return
+      } else {
+        // Get the previous window's reduced value
+        var tempValue = seqOfValues(0).head
+        // If old values exists, then inverse reduce then from previous value
+        if (!oldValues.isEmpty) {
+          tempValue = invReduceF(tempValue, oldValues.reduce(reduceF))
+        }
+        // If new values exists, then reduce them with previous value
+        if (!newValues.isEmpty) {
+          tempValue = reduceF(tempValue, newValues.reduce(reduceF))
+        }
+        tempValue // return
+      }
+    }
+
+    val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues)
+
+    if (filterFunc.isDefined) {
+      Some(mergedValuesRDD.filter(filterFunc.get))
+    } else {
+      Some(mergedValuesRDD)
+    }
+  }
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1f9548bfb85ec952f8d1b35d2549fc31179a2f35
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/ShuffledDStream.scala
@@ -0,0 +1,27 @@
+package spark.streaming.dstream
+
+import spark.{RDD, Partitioner}
+import spark.SparkContext._
+import spark.streaming.{Duration, DStream, Time}
+
+private[streaming]
+class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
+    parent: DStream[(K,V)],
+    createCombiner: V => C,
+    mergeValue: (C, V) => C,
+    mergeCombiner: (C, C) => C,
+    partitioner: Partitioner
+  ) extends DStream [(K,C)] (parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[(K,C)]] = {
+    parent.getOrCompute(validTime) match {
+      case Some(rdd) =>
+        Some(rdd.combineByKey[C](createCombiner, mergeValue, mergeCombiner, partitioner))
+      case None => None
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1408af0afa5018545fd9ec1db61df9182499d095
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
@@ -0,0 +1,77 @@
+package spark.streaming.dstream
+
+import spark.streaming.StreamingContext
+import spark.storage.StorageLevel
+import spark.util.NextIterator
+
+import java.io._
+import java.net.Socket
+
+private[streaming]
+class SocketInputDStream[T: ClassManifest](
+    @transient ssc_ : StreamingContext,
+    host: String,
+    port: Int,
+    bytesToObjects: InputStream => Iterator[T],
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[T](ssc_) {
+
+  def getReceiver(): NetworkReceiver[T] = {
+    new SocketReceiver(host, port, bytesToObjects, storageLevel)
+  }
+}
+
+private[streaming]
+class SocketReceiver[T: ClassManifest](
+    host: String,
+    port: Int,
+    bytesToObjects: InputStream => Iterator[T],
+    storageLevel: StorageLevel
+  ) extends NetworkReceiver[T] {
+
+  lazy protected val blockGenerator = new BlockGenerator(storageLevel)
+
+  override def getLocationPreference = None
+
+  protected def onStart() {
+    logInfo("Connecting to " + host + ":" + port)
+    val socket = new Socket(host, port)
+    logInfo("Connected to " + host + ":" + port)
+    blockGenerator.start()
+    val iterator = bytesToObjects(socket.getInputStream())
+    while(iterator.hasNext) {
+      val obj = iterator.next
+      blockGenerator += obj
+    }
+  }
+
+  protected def onStop() {
+    blockGenerator.stop()
+  }
+
+}
+
+private[streaming]
+object SocketReceiver  {
+
+  /**
+   * This methods translates the data from an inputstream (say, from a socket)
+   * to '\n' delimited strings and returns an iterator to access the strings.
+   */
+  def bytesToLines(inputStream: InputStream): Iterator[String] = {
+    val dataInputStream = new BufferedReader(new InputStreamReader(inputStream, "UTF-8"))
+    new NextIterator[String] {
+      protected override def getNext() = {
+        val nextValue = dataInputStream.readLine()
+        if (nextValue == null) {
+          finished = true
+        }
+        nextValue
+      }
+
+      protected override def close() {
+        dataInputStream.close()
+      }
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..db62955036e9c1a72a5e11571628a97ca858af53
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
@@ -0,0 +1,92 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.Partitioner
+import spark.SparkContext._
+import spark.storage.StorageLevel
+import spark.streaming.{Duration, Time, DStream}
+
+private[streaming]
+class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
+    parent: DStream[(K, V)],
+    updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
+    partitioner: Partitioner,
+    preservePartitioning: Boolean
+  ) extends DStream[(K, S)](parent.ssc) {
+
+  super.persist(StorageLevel.MEMORY_ONLY_SER)
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override val mustCheckpoint = true
+
+  override def compute(validTime: Time): Option[RDD[(K, S)]] = {
+
+    // Try to get the previous state RDD
+    getOrCompute(validTime - slideDuration) match {
+
+      case Some(prevStateRDD) => {    // If previous state RDD exists
+
+        // Try to get the parent RDD
+        parent.getOrCompute(validTime) match {
+          case Some(parentRDD) => {   // If parent RDD exists, then compute as usual
+
+            // Define the function for the mapPartition operation on cogrouped RDD;
+            // first map the cogrouped tuple to tuples of required type,
+            // and then apply the update function
+            val updateFuncLocal = updateFunc
+            val finalFunc = (iterator: Iterator[(K, (Seq[V], Seq[S]))]) => {
+              val i = iterator.map(t => {
+                (t._1, t._2._1, t._2._2.headOption)
+              })
+              updateFuncLocal(i)
+            }
+            val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner)
+            val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning)
+            //logDebug("Generating state RDD for time " + validTime)
+            return Some(stateRDD)
+          }
+          case None => {    // If parent RDD does not exist
+
+            // Re-apply the update function to the old state RDD
+            val updateFuncLocal = updateFunc
+            val finalFunc = (iterator: Iterator[(K, S)]) => {
+              val i = iterator.map(t => (t._1, Seq[V](), Option(t._2)))
+              updateFuncLocal(i)
+            }
+            val stateRDD = prevStateRDD.mapPartitions(finalFunc, preservePartitioning)
+            return Some(stateRDD)
+          }
+        }
+      }
+
+      case None => {    // If previous session RDD does not exist (first input data)
+
+        // Try to get the parent RDD
+        parent.getOrCompute(validTime) match {
+          case Some(parentRDD) => {   // If parent RDD exists, then compute as usual
+
+            // Define the function for the mapPartition operation on grouped RDD;
+            // first map the grouped tuple to tuples of required type,
+            // and then apply the update function
+            val updateFuncLocal = updateFunc
+            val finalFunc = (iterator: Iterator[(K, Seq[V])]) => {
+              updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2, None)))
+            }
+
+            val groupedRDD = parentRDD.groupByKey(partitioner)
+            val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning)
+            //logDebug("Generating state RDD for time " + validTime + " (first)")
+            return Some(sessionRDD)
+          }
+          case None => { // If parent RDD does not exist, then nothing to do!
+            //logDebug("Not generating state RDD (no previous state, no parent)")
+            return None
+          }
+        }
+      }
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..99660d9dee0d02f14cfcd616c2b4c4f440c80c9f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/TransformedDStream.scala
@@ -0,0 +1,19 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.streaming.{Duration, DStream, Time}
+
+private[streaming]
+class TransformedDStream[T: ClassManifest, U: ClassManifest] (
+    parent: DStream[T],
+    transformFunc: (RDD[T], Time) => RDD[U]
+  ) extends DStream[U](parent.ssc) {
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = parent.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[U]] = {
+    parent.getOrCompute(validTime).map(transformFunc(_, validTime))
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c69749886289cd6e49401731f7970628e99115c3
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
@@ -0,0 +1,72 @@
+package spark.streaming.dstream
+
+import spark._
+import spark.streaming._
+import storage.StorageLevel
+
+import twitter4j._
+import twitter4j.auth.BasicAuthorization
+
+/* A stream of Twitter statuses, potentially filtered by one or more keywords.
+*
+* @constructor create a new Twitter stream using the supplied username and password to authenticate.
+* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is
+* such that this may return a sampled subset of all tweets during each interval.
+*/
+private[streaming]
+class TwitterInputDStream(
+    @transient ssc_ : StreamingContext,
+    username: String,
+    password: String,
+    filters: Seq[String],
+    storageLevel: StorageLevel
+  ) extends NetworkInputDStream[Status](ssc_)  {
+
+  override def getReceiver(): NetworkReceiver[Status] = {
+    new TwitterReceiver(username, password, filters, storageLevel)
+  }
+}
+
+private[streaming]
+class TwitterReceiver(
+    username: String,
+    password: String,
+    filters: Seq[String],
+    storageLevel: StorageLevel
+  ) extends NetworkReceiver[Status] {
+
+  var twitterStream: TwitterStream = _
+  lazy val blockGenerator = new BlockGenerator(storageLevel)
+
+  protected override def onStart() {
+    blockGenerator.start()
+    twitterStream = new TwitterStreamFactory()
+      .getInstance(new BasicAuthorization(username, password))
+    twitterStream.addListener(new StatusListener {
+      def onStatus(status: Status) = {
+        blockGenerator += status
+      }
+      // Unimplemented
+      def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {}
+      def onTrackLimitationNotice(i: Int) {}
+      def onScrubGeo(l: Long, l1: Long) {}
+      def onStallWarning(stallWarning: StallWarning) {}
+      def onException(e: Exception) { stopOnError(e) }
+    })
+
+    val query: FilterQuery = new FilterQuery
+    if (filters.size > 0) {
+      query.track(filters.toArray)
+      twitterStream.filter(query)
+    } else {
+      twitterStream.sample()
+    }
+    logInfo("Twitter receiver started")
+  }
+
+  protected override def onStop() {
+    blockGenerator.stop()
+    twitterStream.shutdown()
+    logInfo("Twitter receiver stopped")
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..00bad5da34c7a19da4b98e8a0d20bf7b1ca87b5f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/UnionDStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.dstream
+
+import spark.streaming.{Duration, DStream, Time}
+import spark.RDD
+import collection.mutable.ArrayBuffer
+import spark.rdd.UnionRDD
+
+private[streaming]
+class UnionDStream[T: ClassManifest](parents: Array[DStream[T]])
+  extends DStream[T](parents.head.ssc) {
+
+  if (parents.length == 0) {
+    throw new IllegalArgumentException("Empty array of parents")
+  }
+
+  if (parents.map(_.ssc).distinct.size > 1) {
+    throw new IllegalArgumentException("Array of parents have different StreamingContexts")
+  }
+
+  if (parents.map(_.slideDuration).distinct.size > 1) {
+    throw new IllegalArgumentException("Array of parents have different slide times")
+  }
+
+  override def dependencies = parents.toList
+
+  override def slideDuration: Duration = parents.head.slideDuration
+
+  override def compute(validTime: Time): Option[RDD[T]] = {
+    val rdds = new ArrayBuffer[RDD[T]]()
+    parents.map(_.getOrCompute(validTime)).foreach(_ match {
+      case Some(rdd) => rdds += rdd
+      case None => throw new Exception("Could not generate RDD from a parent for unifying at time " + validTime)
+    })
+    if (rdds.size > 0) {
+      Some(new UnionRDD(ssc.sc, rdds))
+    } else {
+      None
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
new file mode 100644
index 0000000000000000000000000000000000000000..cbf0c88108bfdb39811e597d1f24bb4bc183b789
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/dstream/WindowedDStream.scala
@@ -0,0 +1,40 @@
+package spark.streaming.dstream
+
+import spark.RDD
+import spark.rdd.UnionRDD
+import spark.storage.StorageLevel
+import spark.streaming.{Duration, Interval, Time, DStream}
+
+private[streaming]
+class WindowedDStream[T: ClassManifest](
+    parent: DStream[T],
+    _windowDuration: Duration,
+    _slideDuration: Duration)
+  extends DStream[T](parent.ssc) {
+
+  if (!_windowDuration.isMultipleOf(parent.slideDuration))
+    throw new Exception("The window duration of WindowedDStream (" + _slideDuration + ") " +
+    "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")")
+
+  if (!_slideDuration.isMultipleOf(parent.slideDuration))
+    throw new Exception("The slide duration of WindowedDStream (" + _slideDuration + ") " +
+    "must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")")
+
+  parent.persist(StorageLevel.MEMORY_ONLY_SER)
+
+  def windowDuration: Duration =  _windowDuration
+
+  override def dependencies = List(parent)
+
+  override def slideDuration: Duration = _slideDuration
+
+  override def parentRememberDuration: Duration = rememberDuration + windowDuration
+
+  override def compute(validTime: Time): Option[RDD[T]] = {
+    val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
+    Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
+  }
+}
+
+
+
diff --git a/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b3201d0b28d797c3a8d7e200e19b25fff878730b
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/receivers/ActorReceiver.scala
@@ -0,0 +1,153 @@
+package spark.streaming.receivers
+
+import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }
+import akka.actor.{ actorRef2Scala, ActorRef }
+import akka.actor.{ PossiblyHarmful, OneForOneStrategy }
+
+import spark.storage.StorageLevel
+import spark.streaming.dstream.NetworkReceiver
+
+import java.util.concurrent.atomic.AtomicInteger
+
+/** A helper with set of defaults for supervisor strategy **/
+object ReceiverSupervisorStrategy {
+
+  import akka.util.duration._
+  import akka.actor.SupervisorStrategy._
+
+  val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange =
+    15 millis) {
+    case _: RuntimeException ⇒ Restart
+    case _: Exception ⇒ Escalate
+  }
+}
+
+/**
+ * A receiver trait to be mixed in with your Actor to gain access to
+ * pushBlock API.
+ *
+ * @example {{{
+ * 	class MyActor extends Actor with Receiver{
+ * 		def receive {
+ * 			case anything :String ⇒ pushBlock(anything)
+ * 		}
+ *  }
+ *  //Can be plugged in actorStream as follows
+ *  ssc.actorStream[String](Props(new MyActor),"MyActorReceiver")
+ *
+ * }}}
+ *
+ * @note An important point to note:
+ *       Since Actor may exist outside the spark framework, It is thus user's responsibility
+ *       to ensure the type safety, i.e parametrized type of push block and InputDStream
+ *       should be same.
+ *
+ */
+trait Receiver { self: Actor ⇒
+  def pushBlock[T: ClassManifest](iter: Iterator[T]) {
+    context.parent ! Data(iter)
+  }
+
+  def pushBlock[T: ClassManifest](data: T) {
+    context.parent ! Data(data)
+  }
+
+}
+
+/**
+ * Statistics for querying the supervisor about state of workers
+ */
+case class Statistics(numberOfMsgs: Int,
+  numberOfWorkers: Int,
+  numberOfHiccups: Int,
+  otherInfo: String)
+
+/** Case class to receive data sent by child actors **/
+private[streaming] case class Data[T: ClassManifest](data: T)
+
+/**
+ * Provides Actors as receivers for receiving stream.
+ *
+ * As Actors can also be used to receive data from almost any stream source.
+ * A nice set of abstraction(s) for actors as receivers is already provided for
+ * a few general cases. It is thus exposed as an API where user may come with
+ * his own Actor to run as receiver for Spark Streaming input source.
+ *
+ * This starts a supervisor actor which starts workers and also provides
+ * 	[http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance].
+ * 
+ *  Here's a way to start more supervisor/workers as its children.
+ *
+ * @example {{{
+ * 	context.parent ! Props(new Supervisor)
+ * }}} OR {{{
+ *  context.parent ! Props(new Worker,"Worker")
+ * }}}
+ *
+ *
+ */
+private[streaming] class ActorReceiver[T: ClassManifest](
+  props: Props,
+  name: String,
+  storageLevel: StorageLevel,
+  receiverSupervisorStrategy: SupervisorStrategy)
+  extends NetworkReceiver[T] {
+
+  protected lazy val blocksGenerator: BlockGenerator =
+    new BlockGenerator(storageLevel)
+
+  protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor),
+    "Supervisor" + streamId)
+
+  private class Supervisor extends Actor {
+
+    override val supervisorStrategy = receiverSupervisorStrategy
+    val worker = context.actorOf(props, name)
+    logInfo("Started receiver worker at:" + worker.path)
+
+    val n: AtomicInteger = new AtomicInteger(0)
+    val hiccups: AtomicInteger = new AtomicInteger(0)
+
+    def receive = {
+
+      case Data(iter: Iterator[_]) ⇒ pushBlock(iter.asInstanceOf[Iterator[T]])
+
+      case Data(msg) ⇒
+        blocksGenerator += msg.asInstanceOf[T]
+        n.incrementAndGet
+
+      case props: Props ⇒
+        val worker = context.actorOf(props)
+        logInfo("Started receiver worker at:" + worker.path)
+        sender ! worker
+
+      case (props: Props, name: String) ⇒
+        val worker = context.actorOf(props, name)
+        logInfo("Started receiver worker at:" + worker.path)
+        sender ! worker
+
+      case _: PossiblyHarmful => hiccups.incrementAndGet()
+
+      case _: Statistics ⇒
+        val workers = context.children
+        sender ! Statistics(n.get, workers.size, hiccups.get, workers.mkString("\n"))
+
+    }
+  }
+
+  protected def pushBlock(iter: Iterator[T]) {
+    pushBlock("block-" + streamId + "-" + System.nanoTime(),
+      iter, null, storageLevel)
+  }
+
+  protected def onStart() = {
+    blocksGenerator.start()
+    supervisor
+    logInfo("Supervision tree for receivers initialized at:" + supervisor.path)
+  }
+
+  protected def onStop() = {
+    supervisor ! PoisonPill
+  }
+
+}
diff --git a/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5533c3cf1ef8b5a316aea8b8fa2b9c88f60872fe
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/receivers/ZeroMQReceiver.scala
@@ -0,0 +1,33 @@
+package spark.streaming.receivers
+
+import akka.actor.Actor
+import akka.zeromq._
+
+import spark.Logging
+
+/**
+ * A receiver to subscribe to ZeroMQ stream.
+ */
+private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String,
+  subscribe: Subscribe,
+  bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T])
+  extends Actor with Receiver with Logging {
+
+  override def preStart() = context.system.newSocket(SocketType.Sub, Listener(self),
+    Connect(publisherUrl), subscribe)
+
+  def receive: Receive = {
+
+    case Connecting ⇒ logInfo("connecting ...")
+
+    case m: ZMQMessage ⇒
+      logDebug("Received message for:" + m.firstFrameAsString)
+
+      //We ignore first frame for processing as it is the topic
+      val bytes = m.frames.tail.map(_.payload)
+      pushBlock(bytesToObjects(bytes))
+
+    case Closed ⇒ logInfo("received closed ")
+
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/util/Clock.scala b/streaming/src/main/scala/spark/streaming/util/Clock.scala
new file mode 100644
index 0000000000000000000000000000000000000000..974651f9f6c3363d2b971228a09a7da6decfde1f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/Clock.scala
@@ -0,0 +1,84 @@
+package spark.streaming.util
+
+private[streaming]
+trait Clock {
+  def currentTime(): Long 
+  def waitTillTime(targetTime: Long): Long
+}
+
+private[streaming]
+class SystemClock() extends Clock {
+  
+  val minPollTime = 25L
+  
+  def currentTime(): Long = {
+    System.currentTimeMillis()
+  } 
+  
+  def waitTillTime(targetTime: Long): Long = {
+    var currentTime = 0L
+    currentTime = System.currentTimeMillis()
+    
+    var waitTime = targetTime - currentTime
+    if (waitTime <= 0) {
+      return currentTime
+    }
+    
+    val pollTime = {
+      if (waitTime / 10.0 > minPollTime) {
+        (waitTime / 10.0).toLong
+      } else {
+        minPollTime 
+      }  
+    }
+    
+    
+    while (true) {
+      currentTime = System.currentTimeMillis()
+      waitTime = targetTime - currentTime
+      
+      if (waitTime <= 0) {
+        
+        return currentTime
+      }
+      val sleepTime = 
+        if (waitTime < pollTime) {
+          waitTime
+        } else {
+          pollTime
+        }
+      Thread.sleep(sleepTime)
+    }
+    return -1
+  }
+}
+
+private[streaming]
+class ManualClock() extends Clock {
+  
+  var time = 0L
+
+  def currentTime() = time
+
+  def setTime(timeToSet: Long) = {
+    this.synchronized {
+      time = timeToSet
+      this.notifyAll()
+    }
+  }
+
+  def addToTime(timeToAdd: Long) = {
+    this.synchronized {
+      time += timeToAdd
+      this.notifyAll()
+    } 
+  }
+  def waitTillTime(targetTime: Long): Long = {
+    this.synchronized {
+      while (time < targetTime) {
+        this.wait(100)
+      }      
+    }
+    return currentTime()
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f673e5be15485b839d5f894912d3290eecc21637
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/MasterFailureTest.scala
@@ -0,0 +1,392 @@
+package spark.streaming.util
+
+import spark.{Logging, RDD}
+import spark.streaming._
+import spark.streaming.dstream.ForEachDStream
+import StreamingContext._
+
+import scala.util.Random
+import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+
+import java.io.{File, ObjectInputStream, IOException}
+import java.util.UUID
+
+import com.google.common.io.Files
+
+import org.apache.commons.io.FileUtils
+import org.apache.hadoop.fs.{FileUtil, FileSystem, Path}
+import org.apache.hadoop.conf.Configuration
+
+
+private[streaming]
+object MasterFailureTest extends Logging {
+  initLogging()
+
+  @volatile var killed = false
+  @volatile var killCount = 0
+
+  def main(args: Array[String]) {
+    if (args.size < 2) {
+      println(
+        "Usage: MasterFailureTest <local/HDFS directory> <# batches> [<batch size in milliseconds>]")
+      System.exit(1)
+    }
+    val directory = args(0)
+    val numBatches = args(1).toInt
+    val batchDuration = if (args.size > 2) Milliseconds(args(2).toInt) else Seconds(1)
+
+    println("\n\n========================= MAP TEST =========================\n\n")
+    testMap(directory, numBatches, batchDuration)
+
+    println("\n\n================= UPDATE-STATE-BY-KEY TEST =================\n\n")
+    testUpdateStateByKey(directory, numBatches, batchDuration)
+
+    println("\n\nSUCCESS\n\n")
+  }
+
+  def testMap(directory: String, numBatches: Int, batchDuration: Duration) {
+    // Input: time=1 ==> [ 1 ] , time=2 ==> [ 2 ] , time=3 ==> [ 3 ] , ...
+    val input = (1 to numBatches).map(_.toString).toSeq
+    // Expected output: time=1 ==> [ 1 ] , time=2 ==> [ 2 ] , time=3 ==> [ 3 ] , ...
+    val expectedOutput = (1 to numBatches)
+
+    val operation = (st: DStream[String]) => st.map(_.toInt)
+
+    // Run streaming operation with multiple master failures
+    val output = testOperation(directory, batchDuration, input, operation, expectedOutput)
+
+    logInfo("Expected output, size = " + expectedOutput.size)
+    logInfo(expectedOutput.mkString("[", ",", "]"))
+    logInfo("Output, size = " + output.size)
+    logInfo(output.mkString("[", ",", "]"))
+
+    // Verify whether all the values of the expected output is present
+    // in the output
+    assert(output.distinct.toSet == expectedOutput.toSet)
+  }
+
+
+  def testUpdateStateByKey(directory: String, numBatches: Int, batchDuration: Duration) {
+    // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ...
+    val input = (1 to numBatches).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq
+    // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ...
+    val expectedOutput = (1L to numBatches).map(i => (1L to i).reduce(_ + _)).map(j => ("a", j))
+
+    val operation = (st: DStream[String]) => {
+      val updateFunc = (values: Seq[Long], state: Option[Long]) => {
+        Some(values.foldLeft(0L)(_ + _) + state.getOrElse(0L))
+      }
+      st.flatMap(_.split(" "))
+        .map(x => (x, 1L))
+        .updateStateByKey[Long](updateFunc)
+        .checkpoint(batchDuration * 5)
+    }
+
+    // Run streaming operation with multiple master failures
+    val output = testOperation(directory, batchDuration, input, operation, expectedOutput)
+
+    logInfo("Expected output, size = " + expectedOutput.size + "\n" + expectedOutput)
+    logInfo("Output, size = " + output.size + "\n" + output)
+
+    // Verify whether all the values in the output are among the expected output values
+    output.foreach(o =>
+      assert(expectedOutput.contains(o), "Expected value " + o + " not found")
+    )
+
+    // Verify whether the last expected output value has been generated, there by
+    // confirming that none of the inputs have been missed
+    assert(output.last == expectedOutput.last)
+  }
+
+  /**
+   * Tests stream operation with multiple master failures, and verifies whether the
+   * final set of output values is as expected or not.
+   */
+  def testOperation[T: ClassManifest](
+    directory: String,
+    batchDuration: Duration,
+    input: Seq[String],
+    operation: DStream[String] => DStream[T],
+    expectedOutput: Seq[T]
+  ): Seq[T] = {
+
+    // Just making sure that the expected output does not have duplicates
+    assert(expectedOutput.distinct.toSet == expectedOutput.toSet)
+
+    // Setup the stream computation with the given operation
+    val (ssc, checkpointDir, testDir) = setupStreams(directory, batchDuration, operation)
+
+    // Start generating files in the a different thread
+    val fileGeneratingThread = new FileGeneratingThread(input, testDir, batchDuration.milliseconds)
+    fileGeneratingThread.start()
+
+    // Run the streams and repeatedly kill it until the last expected output
+    // has been generated, or until it has run for twice the expected time
+    val lastExpectedOutput = expectedOutput.last
+    val maxTimeToRun = expectedOutput.size * batchDuration.milliseconds * 2
+    val mergedOutput = runStreams(ssc, lastExpectedOutput, maxTimeToRun)
+
+    // Delete directories
+    fileGeneratingThread.join()
+    val fs = checkpointDir.getFileSystem(new Configuration())
+    fs.delete(checkpointDir, true)
+    fs.delete(testDir, true)
+    logInfo("Finished test after " + killCount + " failures")
+    mergedOutput
+  }
+
+  /**
+   * Sets up the stream computation with the given operation, directory (local or HDFS),
+   * and batch duration. Returns the streaming context and the directory to which
+   * files should be written for testing.
+   */
+  private def setupStreams[T: ClassManifest](
+      directory: String,
+      batchDuration: Duration,
+      operation: DStream[String] => DStream[T]
+    ): (StreamingContext, Path, Path) = {
+    // Reset all state
+    reset()
+
+    // Create the directories for this test
+    val uuid = UUID.randomUUID().toString
+    val rootDir = new Path(directory, uuid)
+    val fs = rootDir.getFileSystem(new Configuration())
+    val checkpointDir = new Path(rootDir, "checkpoint")
+    val testDir = new Path(rootDir, "test")
+    fs.mkdirs(checkpointDir)
+    fs.mkdirs(testDir)
+
+    // Setup the streaming computation with the given operation
+    System.clearProperty("spark.driver.port")
+    var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map())
+    ssc.checkpoint(checkpointDir.toString)
+    val inputStream = ssc.textFileStream(testDir.toString)
+    val operatedStream = operation(inputStream)
+    val outputStream = new TestOutputStream(operatedStream)
+    ssc.registerOutputStream(outputStream)
+    (ssc, checkpointDir, testDir)
+  }
+
+
+  /**
+   * Repeatedly starts and kills the streaming context until timed out or
+   * the last expected output is generated. Finally, return
+   */
+  private def runStreams[T: ClassManifest](
+      ssc_ : StreamingContext,
+      lastExpectedOutput: T,
+      maxTimeToRun: Long
+   ): Seq[T] = {
+
+    var ssc = ssc_
+    var totalTimeRan = 0L
+    var isLastOutputGenerated = false
+    var isTimedOut = false
+    val mergedOutput = new ArrayBuffer[T]()
+    val checkpointDir = ssc.checkpointDir
+    var batchDuration = ssc.graph.batchDuration
+
+    while(!isLastOutputGenerated && !isTimedOut) {
+      // Get the output buffer
+      val outputBuffer = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[T]].output
+      def output = outputBuffer.flatMap(x => x)
+
+      // Start the thread to kill the streaming after some time
+      killed = false
+      val killingThread = new KillingThread(ssc, batchDuration.milliseconds * 10)
+      killingThread.start()
+
+      var timeRan = 0L
+      try {
+        // Start the streaming computation and let it run while ...
+        // (i) StreamingContext has not been shut down yet
+        // (ii) The last expected output has not been generated yet
+        // (iii) Its not timed out yet
+        System.clearProperty("spark.streaming.clock")
+        System.clearProperty("spark.driver.port")
+        ssc.start()
+        val startTime = System.currentTimeMillis()
+        while (!killed && !isLastOutputGenerated && !isTimedOut) {
+          Thread.sleep(100)
+          timeRan = System.currentTimeMillis() - startTime
+          isLastOutputGenerated = (!output.isEmpty && output.last == lastExpectedOutput)
+          isTimedOut = (timeRan + totalTimeRan > maxTimeToRun)
+        }
+      } catch {
+        case e: Exception => logError("Error running streaming context", e)
+      }
+      if (killingThread.isAlive) killingThread.interrupt()
+      ssc.stop()
+
+      logInfo("Has been killed = " + killed)
+      logInfo("Is last output generated = " + isLastOutputGenerated)
+      logInfo("Is timed out = " + isTimedOut)
+
+      // Verify whether the output of each batch has only one element or no element
+      // and then merge the new output with all the earlier output
+      mergedOutput ++= output
+      totalTimeRan += timeRan
+      logInfo("New output = " + output)
+      logInfo("Merged output = " + mergedOutput)
+      logInfo("Time ran = " + timeRan)
+      logInfo("Total time ran = " + totalTimeRan)
+
+      if (!isLastOutputGenerated && !isTimedOut) {
+        val sleepTime = Random.nextInt(batchDuration.milliseconds.toInt * 10)
+        logInfo(
+          "\n-------------------------------------------\n" +
+            "   Restarting stream computation in " + sleepTime + " ms   " +
+            "\n-------------------------------------------\n"
+        )
+        Thread.sleep(sleepTime)
+        // Recreate the streaming context from checkpoint
+        ssc = new StreamingContext(checkpointDir)
+      }
+    }
+    mergedOutput
+  }
+
+  /**
+   * Verifies the output value are the same as expected. Since failures can lead to
+   * a batch being processed twice, a batches output may appear more than once
+   * consecutively. To avoid getting confused with those, we eliminate consecutive
+   * duplicate batch outputs of values from the `output`. As a result, the
+   * expected output should not have consecutive batches with the same values as output.
+   */
+  private def verifyOutput[T: ClassManifest](output: Seq[T], expectedOutput: Seq[T]) {
+    // Verify whether expected outputs do not consecutive batches with same output
+    for (i <- 0 until expectedOutput.size - 1) {
+      assert(expectedOutput(i) != expectedOutput(i+1),
+        "Expected output has consecutive duplicate sequence of values")
+    }
+
+    // Log the output
+    println("Expected output, size = " + expectedOutput.size)
+    println(expectedOutput.mkString("[", ",", "]"))
+    println("Output, size = " + output.size)
+    println(output.mkString("[", ",", "]"))
+
+    // Match the output with the expected output
+    output.foreach(o =>
+      assert(expectedOutput.contains(o), "Expected value " + o + " not found")
+    )
+  }
+
+  /** Resets counter to prepare for the test */
+  private def reset() {
+    killed = false
+    killCount = 0
+  }
+}
+
+/**
+ * This is a output stream just for testing. All the output is collected into a
+ * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
+ */
+private[streaming]
+class TestOutputStream[T: ClassManifest](
+    parent: DStream[T],
+    val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]]
+  ) extends ForEachDStream[T](
+    parent,
+    (rdd: RDD[T], t: Time) => {
+      val collected = rdd.collect()
+      output += collected
+    }
+  ) {
+
+  // This is to clear the output buffer every it is read from a checkpoint
+  @throws(classOf[IOException])
+  private def readObject(ois: ObjectInputStream) {
+    ois.defaultReadObject()
+    output.clear()
+  }
+}
+
+
+/**
+ * Thread to kill streaming context after a random period of time.
+ */
+private[streaming]
+class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging {
+  initLogging()
+
+  override def run() {
+    try {
+      // If it is the first killing, then allow the first checkpoint to be created
+      var minKillWaitTime = if (MasterFailureTest.killCount == 0) 5000 else 2000
+      val killWaitTime = minKillWaitTime + math.abs(Random.nextLong % maxKillWaitTime)
+      logInfo("Kill wait time = " + killWaitTime)
+      Thread.sleep(killWaitTime)
+      logInfo(
+        "\n---------------------------------------\n" +
+          "Killing streaming context after " + killWaitTime + " ms" +
+          "\n---------------------------------------\n"
+      )
+      if (ssc != null) {
+        ssc.stop()
+        MasterFailureTest.killed = true
+        MasterFailureTest.killCount += 1
+      }
+      logInfo("Killing thread finished normally")
+    } catch {
+      case ie: InterruptedException => logInfo("Killing thread interrupted")
+      case e: Exception => logWarning("Exception in killing thread", e)
+    }
+
+  }
+}
+
+
+/**
+ * Thread to generate input files periodically with the desired text.
+ */
+private[streaming]
+class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
+  extends Thread with Logging {
+  initLogging()
+
+  override def run() {
+    val localTestDir = Files.createTempDir()
+    var fs = testDir.getFileSystem(new Configuration())
+    val maxTries = 3
+    try {
+      Thread.sleep(5000) // To make sure that all the streaming context has been set up
+      for (i <- 0 until input.size) {
+        // 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)
+        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)
+	    done = true
+	  } catch {
+	    case ioe: IOException => { 
+              fs = testDir.getFileSystem(new Configuration()) 
+              logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe)
+	    }
+	  }
+        }
+	if (!done) 
+          logError("Could not generate file " + hadoopFile)
+        else 
+          logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis)
+        Thread.sleep(interval)
+        localFile.delete()
+      }
+      logInfo("File generating thread finished normally")
+    } catch {
+      case ie: InterruptedException => logInfo("File generating thread interrupted")
+      case e: Exception => logWarning("File generating in killing thread", e)
+    } finally {
+      fs.close()
+    }
+  }
+}
+
+
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
new file mode 100644
index 0000000000000000000000000000000000000000..03749d4a94457a3a0ff5f1f43123278a96fd53a5
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextHelper.scala
@@ -0,0 +1,98 @@
+package spark.streaming.util
+
+import spark.SparkContext
+import spark.SparkContext._
+import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
+import scala.collection.JavaConversions.mapAsScalaMap
+
+object RawTextHelper {
+
+  /** 
+   * Splits lines and counts the words in them using specialized object-to-long hashmap 
+   * (to avoid boxing-unboxing overhead of Long in java/scala HashMap)
+   */
+  def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = {
+    val map = new OLMap[String]
+    var i = 0
+    var j = 0
+    while (iter.hasNext) {
+      val s = iter.next()
+      i = 0
+      while (i < s.length) {
+        j = i
+        while (j < s.length && s.charAt(j) != ' ') {
+          j += 1
+        }
+        if (j > i) {
+          val w = s.substring(i, j)
+          val c = map.getLong(w)
+          map.put(w, c + 1)
+        }
+        i = j
+        while (i < s.length && s.charAt(i) == ' ') {
+          i += 1
+        }
+      }
+    }
+    map.toIterator.map{case (k, v) => (k, v)}
+  }
+
+  /** 
+   * Gets the top k words in terms of word counts. Assumes that each word exists only once
+   * in the `data` iterator (that is, the counts have been reduced).
+   */
+  def topK(data: Iterator[(String, Long)], k: Int): Iterator[(String, Long)] = {
+    val taken = new Array[(String, Long)](k)
+    
+    var i = 0
+    var len = 0
+    var done = false
+    var value: (String, Long) = null
+    var swap: (String, Long) = null
+    var count = 0
+
+    while(data.hasNext) {
+      value = data.next
+      if (value != null) {
+        count += 1
+        if (len == 0) {
+          taken(0) = value
+          len = 1
+        } else if (len < k || value._2 > taken(len - 1)._2) {
+          if (len < k) {
+            len += 1
+          }
+          taken(len - 1) = value
+          i = len - 1
+          while(i > 0 && taken(i - 1)._2 < taken(i)._2) {
+            swap = taken(i)
+            taken(i) = taken(i-1)
+            taken(i - 1) = swap
+            i -= 1
+          }
+        }
+      }
+    }
+    return taken.toIterator  
+  }
+ 
+  /**
+   * Warms up the SparkContext in master and slave by running tasks to force JIT kick in
+   * before real workload starts.
+   */
+  def warmUp(sc: SparkContext) {
+    for(i <- 0 to 1) {
+      sc.parallelize(1 to 200000, 1000)
+        .map(_ % 1331).map(_.toString)
+        .mapPartitions(splitAndCountPartitions).reduceByKey(_ + _, 10)
+        .count()
+    }
+  }
+  
+  def add(v1: Long, v2: Long) = (v1 + v2) 
+
+  def subtract(v1: Long, v2: Long) = (v1 - v2) 
+
+  def max(v1: Long, v2: Long) = math.max(v1, v2) 
+}
+
diff --git a/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
new file mode 100644
index 0000000000000000000000000000000000000000..d8b987ec860bbcd3ea4d08d8a02a7bd213c7fb9f
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/RawTextSender.scala
@@ -0,0 +1,60 @@
+package spark.streaming.util
+
+import java.nio.ByteBuffer
+import spark.util.{RateLimitedOutputStream, IntParam}
+import java.net.ServerSocket
+import spark.{Logging, KryoSerializer}
+import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+import io.Source
+import java.io.IOException
+
+/**
+ * A helper program that sends blocks of Kryo-serialized text strings out on a socket at a
+ * specified rate. Used to feed data into RawInputDStream.
+ */
+object RawTextSender extends Logging {
+  def main(args: Array[String]) {
+    if (args.length != 4) {
+      System.err.println("Usage: RawTextSender <port> <file> <blockSize> <bytesPerSec>")
+      System.exit(1)
+    }
+    // Parse the arguments using a pattern match
+    val Array(IntParam(port), file, IntParam(blockSize), IntParam(bytesPerSec)) = args
+
+    // Repeat the input data multiple times to fill in a buffer
+    val lines = Source.fromFile(file).getLines().toArray
+    val bufferStream = new FastByteArrayOutputStream(blockSize + 1000)
+    val ser = new KryoSerializer().newInstance()
+    val serStream = ser.serializeStream(bufferStream)
+    var i = 0
+    while (bufferStream.position < blockSize) {
+      serStream.writeObject(lines(i))
+      i = (i + 1) % lines.length
+    }
+    bufferStream.trim()
+    val array = bufferStream.array
+
+    val countBuf = ByteBuffer.wrap(new Array[Byte](4))
+    countBuf.putInt(array.length)
+    countBuf.flip()
+
+    val serverSocket = new ServerSocket(port)
+    logInfo("Listening on port " + port)
+
+    while (true) {
+      val socket = serverSocket.accept()
+      logInfo("Got a new connection")
+      val out = new RateLimitedOutputStream(socket.getOutputStream, bytesPerSec)
+      try {
+        while (true) {
+          out.write(countBuf.array)
+          out.write(array)
+        }
+      } catch {
+        case e: IOException =>
+          logError("Client disconnected")
+          socket.close()
+      }
+    }
+  }
+}
diff --git a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
new file mode 100644
index 0000000000000000000000000000000000000000..8e10276deb9056301f98712637208db3ce2a74b7
--- /dev/null
+++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
@@ -0,0 +1,77 @@
+package spark.streaming.util
+
+private[streaming]
+class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) {
+  
+  private val minPollTime = 25L
+  
+  private val pollTime = {
+    if (period / 10.0 > minPollTime) {
+      (period / 10.0).toLong
+    } else {
+      minPollTime
+    }  
+  }
+  
+  private val thread = new Thread() {
+    override def run() { loop }    
+  }
+  
+  private var nextTime = 0L
+
+  def getStartTime(): Long = {
+    (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period
+  }
+
+  def getRestartTime(originalStartTime: Long): Long = {
+    val gap = clock.currentTime - originalStartTime
+    (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime
+  }
+
+  def start(startTime: Long): Long = {
+    nextTime = startTime
+    thread.start()
+    nextTime
+  }
+
+  def start(): Long = {
+    start(getStartTime())
+  }
+
+  def stop() {
+    thread.interrupt() 
+  }
+  
+  private def loop() {
+    try {
+      while (true) {
+        clock.waitTillTime(nextTime)
+        callback(nextTime)
+        nextTime += period
+      }
+      
+    } catch {
+      case e: InterruptedException =>
+    }
+  }
+}
+
+private[streaming]
+object RecurringTimer {
+  
+  def main(args: Array[String]) {
+    var lastRecurTime = 0L
+    val period = 1000
+    
+    def onRecur(time: Long) {
+      val currentTime = System.currentTimeMillis()
+      println("" + currentTime + ": " + (currentTime - lastRecurTime))
+      lastRecurTime = currentTime
+    }
+    val timer = new  RecurringTimer(new SystemClock(), period, onRecur)
+    timer.start()
+    Thread.sleep(30 * 1000)
+    timer.stop()
+  }
+}
+
diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java
new file mode 100644
index 0000000000000000000000000000000000000000..3bed500f73e84548a2f470a414ea68cfbb3df1ac
--- /dev/null
+++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java
@@ -0,0 +1,1283 @@
+package spark.streaming;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.io.Files;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import scala.Tuple2;
+import spark.HashPartitioner;
+import spark.api.java.JavaPairRDD;
+import spark.api.java.JavaRDD;
+import spark.api.java.JavaRDDLike;
+import spark.api.java.JavaPairRDD;
+import spark.api.java.JavaSparkContext;
+import spark.api.java.function.*;
+import spark.storage.StorageLevel;
+import spark.streaming.api.java.JavaDStream;
+import spark.streaming.api.java.JavaPairDStream;
+import spark.streaming.api.java.JavaStreamingContext;
+import spark.streaming.JavaTestUtils;
+import spark.streaming.JavaCheckpointTestUtils;
+import spark.streaming.dstream.KafkaPartitionKey;
+import spark.streaming.InputStreamsSuite;
+
+import java.io.*;
+import java.util.*;
+
+import akka.actor.Props;
+import akka.zeromq.Subscribe;
+
+
+
+// The test suite itself is Serializable so that anonymous Function implementations can be
+// serialized, as an alternative to converting these anonymous classes to static inner classes;
+// see http://stackoverflow.com/questions/758570/.
+public class JavaAPISuite implements Serializable {
+  private transient JavaStreamingContext ssc;
+
+  @Before
+  public void setUp() {
+      System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock");
+      ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+    ssc.checkpoint("checkpoint");
+  }
+
+  @After
+  public void tearDown() {
+    ssc.stop();
+    ssc = null;
+
+    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+    System.clearProperty("spark.driver.port");
+  }
+
+  @Test
+  public void testCount() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3,4),
+        Arrays.asList(3,4,5),
+        Arrays.asList(3));
+
+    List<List<Long>> expected = Arrays.asList(
+        Arrays.asList(4L),
+        Arrays.asList(3L),
+        Arrays.asList(1L));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream count = stream.count();
+    JavaTestUtils.attachTestOutputStream(count);
+    List<List<Long>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testMap() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("hello", "world"),
+        Arrays.asList("goodnight", "moon"));
+
+   List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(5,5),
+        Arrays.asList(9,4));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+        @Override
+        public Integer call(String s) throws Exception {
+          return s.length();
+        }
+    });
+    JavaTestUtils.attachTestOutputStream(letterCount);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testWindow() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6,1,2,3),
+        Arrays.asList(7,8,9,4,5,6),
+        Arrays.asList(7,8,9));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream windowed = stream.window(new Duration(2000));
+    JavaTestUtils.attachTestOutputStream(windowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testWindowWithSlideDuration() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9),
+        Arrays.asList(10,11,12),
+        Arrays.asList(13,14,15),
+        Arrays.asList(16,17,18));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,2,3,4,5,6),
+        Arrays.asList(1,2,3,4,5,6,7,8,9,10,11,12),
+        Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18),
+        Arrays.asList(13,14,15,16,17,18));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000));
+    JavaTestUtils.attachTestOutputStream(windowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 8, 4);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testFilter() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red socks"));
+
+    List<List<String>> expected = Arrays.asList(
+        Arrays.asList("giants"),
+        Arrays.asList("yankees"));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream filtered = stream.filter(new Function<String, Boolean>() {
+      @Override
+      public Boolean call(String s) throws Exception {
+        return s.contains("a");
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(filtered);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testGlom() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red socks"));
+
+    List<List<List<String>>> expected = Arrays.asList(
+        Arrays.asList(Arrays.asList("giants", "dodgers")),
+        Arrays.asList(Arrays.asList("yankees", "red socks")));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream glommed = stream.glom();
+    JavaTestUtils.attachTestOutputStream(glommed);
+    List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testMapPartitions() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red socks"));
+
+    List<List<String>> expected = Arrays.asList(
+        Arrays.asList("GIANTSDODGERS"),
+        Arrays.asList("YANKEESRED SOCKS"));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream mapped = stream.mapPartitions(new FlatMapFunction<Iterator<String>, String>() {
+      @Override
+      public Iterable<String> call(Iterator<String> in) {
+        String out = "";
+        while (in.hasNext()) {
+          out = out + in.next().toUpperCase();
+        }
+        return Lists.newArrayList(out);
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(mapped);
+    List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  private class IntegerSum extends Function2<Integer, Integer, Integer> {
+    @Override
+    public Integer call(Integer i1, Integer i2) throws Exception {
+      return i1 + i2;
+    }
+  }
+
+  private class IntegerDifference extends Function2<Integer, Integer, Integer> {
+    @Override
+    public Integer call(Integer i1, Integer i2) throws Exception {
+      return i1 - i2;
+    }
+  }
+
+  @Test
+  public void testReduce() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(6),
+        Arrays.asList(15),
+        Arrays.asList(24));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream reduced = stream.reduce(new IntegerSum());
+    JavaTestUtils.attachTestOutputStream(reduced);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testReduceByWindow() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(6),
+        Arrays.asList(21),
+        Arrays.asList(39),
+        Arrays.asList(24));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(),
+        new IntegerDifference(), new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reducedWindowed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testQueueStream() {
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
+    JavaRDD<Integer> rdd1 = ssc.sc().parallelize(Arrays.asList(1,2,3));
+    JavaRDD<Integer> rdd2 = ssc.sc().parallelize(Arrays.asList(4,5,6));
+    JavaRDD<Integer> rdd3 = ssc.sc().parallelize(Arrays.asList(7,8,9));
+
+    LinkedList<JavaRDD<Integer>> rdds = Lists.newLinkedList();
+    rdds.add(rdd1);
+    rdds.add(rdd2);
+    rdds.add(rdd3);
+
+    JavaDStream<Integer> stream = ssc.queueStream(rdds);
+    JavaTestUtils.attachTestOutputStream(stream);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testTransform() {
+    List<List<Integer>> inputData = Arrays.asList(
+        Arrays.asList(1,2,3),
+        Arrays.asList(4,5,6),
+        Arrays.asList(7,8,9));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(3,4,5),
+        Arrays.asList(6,7,8),
+        Arrays.asList(9,10,11));
+
+    JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream<Integer> transformed =
+        stream.transform(new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+      @Override
+      public JavaRDD<Integer> call(JavaRDD<Integer> in) throws Exception {
+        return in.map(new Function<Integer, Integer>() {
+          @Override
+          public Integer call(Integer i) throws Exception {
+            return i + 2;
+          }
+        });
+      }});
+    JavaTestUtils.attachTestOutputStream(transformed);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testFlatMap() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("go", "giants"),
+        Arrays.asList("boo", "dodgers"),
+        Arrays.asList("athletics"));
+
+    List<List<String>> expected = Arrays.asList(
+        Arrays.asList("g","o","g","i","a","n","t","s"),
+        Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"),
+        Arrays.asList("a","t","h","l","e","t","i","c","s"));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
+      @Override
+      public Iterable<String> call(String x) {
+        return Lists.newArrayList(x.split("(?!^)"));
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  @Test
+  public void testPairFlatMap() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants"),
+        Arrays.asList("dodgers"),
+        Arrays.asList("athletics"));
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<Integer, String>(6, "g"),
+            new Tuple2<Integer, String>(6, "i"),
+            new Tuple2<Integer, String>(6, "a"),
+            new Tuple2<Integer, String>(6, "n"),
+            new Tuple2<Integer, String>(6, "t"),
+            new Tuple2<Integer, String>(6, "s")),
+        Arrays.asList(
+            new Tuple2<Integer, String>(7, "d"),
+            new Tuple2<Integer, String>(7, "o"),
+            new Tuple2<Integer, String>(7, "d"),
+            new Tuple2<Integer, String>(7, "g"),
+            new Tuple2<Integer, String>(7, "e"),
+            new Tuple2<Integer, String>(7, "r"),
+            new Tuple2<Integer, String>(7, "s")),
+        Arrays.asList(
+            new Tuple2<Integer, String>(9, "a"),
+            new Tuple2<Integer, String>(9, "t"),
+            new Tuple2<Integer, String>(9, "h"),
+            new Tuple2<Integer, String>(9, "l"),
+            new Tuple2<Integer, String>(9, "e"),
+            new Tuple2<Integer, String>(9, "t"),
+            new Tuple2<Integer, String>(9, "i"),
+            new Tuple2<Integer, String>(9, "c"),
+            new Tuple2<Integer, String>(9, "s")));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction<String, Integer, String>() {
+      @Override
+      public Iterable<Tuple2<Integer, String>> call(String in) throws Exception {
+        List<Tuple2<Integer, String>> out = Lists.newArrayList();
+        for (String letter: in.split("(?!^)")) {
+          out.add(new Tuple2<Integer, String>(in.length(), letter));
+        }
+        return out;
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testUnion() {
+    List<List<Integer>> inputData1 = Arrays.asList(
+        Arrays.asList(1,1),
+        Arrays.asList(2,2),
+        Arrays.asList(3,3));
+
+    List<List<Integer>> inputData2 = Arrays.asList(
+        Arrays.asList(4,4),
+        Arrays.asList(5,5),
+        Arrays.asList(6,6));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(1,1,4,4),
+        Arrays.asList(2,2,5,5),
+        Arrays.asList(3,3,6,6));
+
+    JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
+    JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
+
+    JavaDStream unioned = stream1.union(stream2);
+    JavaTestUtils.attachTestOutputStream(unioned);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assertOrderInvariantEquals(expected, result);
+  }
+
+  /*
+   * Performs an order-invariant comparison of lists representing two RDD streams. This allows
+   * us to account for ordering variation within individual RDD's which occurs during windowing.
+   */
+  public static <T extends Comparable> void assertOrderInvariantEquals(
+      List<List<T>> expected, List<List<T>> actual) {
+    for (List<T> list: expected) {
+      Collections.sort(list);
+    }
+    for (List<T> list: actual) {
+      Collections.sort(list);
+    }
+    Assert.assertEquals(expected, actual);
+  }
+
+
+  // PairDStream Functions
+  @Test
+  public void testPairFilter() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("giants", "dodgers"),
+        Arrays.asList("yankees", "red socks"));
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<String, Integer>("giants", 6)),
+        Arrays.asList(new Tuple2<String, Integer>("yankees", 7)));
+
+    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = stream.map(
+        new PairFunction<String, String, Integer>() {
+          @Override
+          public Tuple2 call(String in) throws Exception {
+            return new Tuple2<String, Integer>(in, in.length());
+          }
+        });
+
+    JavaPairDStream<String, Integer> filtered = pairStream.filter(
+        new Function<Tuple2<String, Integer>, Boolean>() {
+      @Override
+      public Boolean call(Tuple2<String, Integer> in) throws Exception {
+        return in._1().contains("a");
+      }
+    });
+    JavaTestUtils.attachTestOutputStream(filtered);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  List<List<Tuple2<String, String>>> stringStringKVStream = Arrays.asList(
+      Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+          new Tuple2<String, String>("california", "giants"),
+          new Tuple2<String, String>("new york", "yankees"),
+          new Tuple2<String, String>("new york", "mets")),
+      Arrays.asList(new Tuple2<String, String>("california", "sharks"),
+          new Tuple2<String, String>("california", "ducks"),
+          new Tuple2<String, String>("new york", "rangers"),
+          new Tuple2<String, String>("new york", "islanders")));
+
+  List<List<Tuple2<String, Integer>>> stringIntKVStream = Arrays.asList(
+      Arrays.asList(
+          new Tuple2<String, Integer>("california", 1),
+          new Tuple2<String, Integer>("california", 3),
+          new Tuple2<String, Integer>("new york", 4),
+          new Tuple2<String, Integer>("new york", 1)),
+      Arrays.asList(
+          new Tuple2<String, Integer>("california", 5),
+          new Tuple2<String, Integer>("california", 5),
+          new Tuple2<String, Integer>("new york", 3),
+          new Tuple2<String, Integer>("new york", 1)));
+
+  @Test
+  public void testPairMap() { // Maps pair -> pair of different type
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+                new Tuple2<Integer, String>(1, "california"),
+                new Tuple2<Integer, String>(3, "california"),
+                new Tuple2<Integer, String>(4, "new york"),
+                new Tuple2<Integer, String>(1, "new york")),
+        Arrays.asList(
+                new Tuple2<Integer, String>(5, "california"),
+                new Tuple2<Integer, String>(5, "california"),
+                new Tuple2<Integer, String>(3, "new york"),
+                new Tuple2<Integer, String>(1, "new york")));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> reversed = pairStream.map(
+        new PairFunction<Tuple2<String, Integer>, Integer, String>() {
+          @Override
+          public Tuple2<Integer, String> call(Tuple2<String, Integer> in) throws Exception {
+            return in.swap();
+          }
+    });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairMapPartitions() { // Maps pair -> pair of different type
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<Integer, String>(1, "california"),
+            new Tuple2<Integer, String>(3, "california"),
+            new Tuple2<Integer, String>(4, "new york"),
+            new Tuple2<Integer, String>(1, "new york")),
+        Arrays.asList(
+            new Tuple2<Integer, String>(5, "california"),
+            new Tuple2<Integer, String>(5, "california"),
+            new Tuple2<Integer, String>(3, "new york"),
+            new Tuple2<Integer, String>(1, "new york")));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> reversed = pairStream.mapPartitions(
+        new PairFlatMapFunction<Iterator<Tuple2<String, Integer>>, Integer, String>() {
+          @Override
+          public Iterable<Tuple2<Integer, String>> call(Iterator<Tuple2<String, Integer>> in) throws Exception {
+            LinkedList<Tuple2<Integer, String>> out = new LinkedList<Tuple2<Integer, String>>();
+            while (in.hasNext()) {
+              Tuple2<String, Integer> next = in.next();
+              out.add(next.swap());
+            }
+            return out;
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairMap2() { // Maps pair -> single
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Integer>> expected = Arrays.asList(
+            Arrays.asList(1, 3, 4, 1),
+            Arrays.asList(5, 5, 3, 1));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaDStream<Integer> reversed = pairStream.map(
+            new Function<Tuple2<String, Integer>, Integer>() {
+              @Override
+              public Integer call(Tuple2<String, Integer> in) throws Exception {
+                return in._2();
+              }
+            });
+
+    JavaTestUtils.attachTestOutputStream(reversed);
+    List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair
+    List<List<Tuple2<String, Integer>>> inputData = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Integer>("hi", 1),
+            new Tuple2<String, Integer>("ho", 2)),
+        Arrays.asList(
+            new Tuple2<String, Integer>("hi", 1),
+            new Tuple2<String, Integer>("ho", 2)));
+
+    List<List<Tuple2<Integer, String>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<Integer, String>(1, "h"),
+            new Tuple2<Integer, String>(1, "i"),
+            new Tuple2<Integer, String>(2, "h"),
+            new Tuple2<Integer, String>(2, "o")),
+        Arrays.asList(
+            new Tuple2<Integer, String>(1, "h"),
+            new Tuple2<Integer, String>(1, "i"),
+            new Tuple2<Integer, String>(2, "h"),
+            new Tuple2<Integer, String>(2, "o")));
+
+    JavaDStream<Tuple2<String, Integer>> stream =
+        JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaPairDStream<Integer, String> flatMapped = pairStream.flatMap(
+        new PairFlatMapFunction<Tuple2<String, Integer>, Integer, String>() {
+          @Override
+          public Iterable<Tuple2<Integer, String>> call(Tuple2<String, Integer> in) throws Exception {
+            List<Tuple2<Integer, String>> out = new LinkedList<Tuple2<Integer, String>>();
+            for (Character s : in._1().toCharArray()) {
+              out.add(new Tuple2<Integer, String>(in._2(), s.toString()));
+            }
+            return out;
+          }
+        });
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairGroupByKey() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, List<String>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, List<String>>("california", Arrays.asList("dodgers", "giants")),
+            new Tuple2<String, List<String>>("new york", Arrays.asList("yankees", "mets"))),
+        Arrays.asList(
+            new Tuple2<String, List<String>>("california", Arrays.asList("sharks", "ducks")),
+            new Tuple2<String, List<String>>("new york", Arrays.asList("rangers", "islanders"))));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, List<String>> grouped = pairStream.groupByKey();
+    JavaTestUtils.attachTestOutputStream(grouped);
+    List<List<Tuple2<String, List<String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairReduceByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Integer>("california", 4),
+            new Tuple2<String, Integer>("new york", 5)),
+        Arrays.asList(
+            new Tuple2<String, Integer>("california", 10),
+            new Tuple2<String, Integer>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduced = pairStream.reduceByKey(new IntegerSum());
+
+    JavaTestUtils.attachTestOutputStream(reduced);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCombineByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Integer>("california", 4),
+            new Tuple2<String, Integer>("new york", 5)),
+        Arrays.asList(
+            new Tuple2<String, Integer>("california", 10),
+            new Tuple2<String, Integer>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> combined = pairStream.<Integer>combineByKey(
+        new Function<Integer, Integer>() {
+        @Override
+          public Integer call(Integer i) throws Exception {
+            return i;
+          }
+        }, new IntegerSum(), new IntegerSum(), new HashPartitioner(2));
+
+    JavaTestUtils.attachTestOutputStream(combined);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCountByValue() {
+    List<List<String>> inputData = Arrays.asList(
+      Arrays.asList("hello", "world"),
+      Arrays.asList("hello", "moon"),
+      Arrays.asList("hello"));
+
+    List<List<Tuple2<String, Long>>> expected = Arrays.asList(
+      Arrays.asList(
+              new Tuple2<String, Long>("hello", 1L),
+              new Tuple2<String, Long>("world", 1L)),
+      Arrays.asList(
+              new Tuple2<String, Long>("hello", 1L),
+              new Tuple2<String, Long>("moon", 1L)),
+      Arrays.asList(
+              new Tuple2<String, Long>("hello", 1L)));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Long> counted = stream.countByValue();
+    JavaTestUtils.attachTestOutputStream(counted);
+    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testGroupByKeyAndWindow() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, List<Integer>>>> expected = Arrays.asList(
+      Arrays.asList(
+        new Tuple2<String, List<Integer>>("california", Arrays.asList(1, 3)),
+        new Tuple2<String, List<Integer>>("new york", Arrays.asList(1, 4))
+      ),
+      Arrays.asList(
+        new Tuple2<String, List<Integer>>("california", Arrays.asList(1, 3, 5, 5)),
+        new Tuple2<String, List<Integer>>("new york", Arrays.asList(1, 1, 3, 4))
+      ),
+      Arrays.asList(
+        new Tuple2<String, List<Integer>>("california", Arrays.asList(5, 5)),
+        new Tuple2<String, List<Integer>>("new york", Arrays.asList(1, 3))
+      )
+    );
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, List<Integer>> groupWindowed =
+        pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(groupWindowed);
+    List<List<Tuple2<String, List<Integer>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    assert(result.size() == expected.size());
+    for (int i = 0; i < result.size(); i++) {
+      assert(convert(result.get(i)).equals(convert(expected.get(i))));
+    }
+  }
+
+  private HashSet<Tuple2<String, HashSet<Integer>>> convert(List<Tuple2<String, List<Integer>>> listOfTuples) {
+    List<Tuple2<String, HashSet<Integer>>> newListOfTuples = new ArrayList<Tuple2<String, HashSet<Integer>>>();
+    for (Tuple2<String, List<Integer>> tuple: listOfTuples) {
+      newListOfTuples.add(convert(tuple));
+    }
+    return new HashSet<Tuple2<String, HashSet<Integer>>>(newListOfTuples);
+  }
+
+  private Tuple2<String, HashSet<Integer>> convert(Tuple2<String, List<Integer>> tuple) {
+    return new Tuple2<String, HashSet<Integer>>(tuple._1(), new HashSet<Integer>(tuple._2()));
+  }
+
+  @Test
+  public void testReduceByKeyAndWindow() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<String, Integer>("california", 4),
+            new Tuple2<String, Integer>("new york", 5)),
+        Arrays.asList(new Tuple2<String, Integer>("california", 14),
+            new Tuple2<String, Integer>("new york", 9)),
+        Arrays.asList(new Tuple2<String, Integer>("california", 10),
+            new Tuple2<String, Integer>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduceWindowed =
+        pairStream.reduceByKeyAndWindow(new IntegerSum(), new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reduceWindowed);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testUpdateStateByKey() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<String, Integer>("california", 4),
+            new Tuple2<String, Integer>("new york", 5)),
+        Arrays.asList(new Tuple2<String, Integer>("california", 14),
+            new Tuple2<String, Integer>("new york", 9)),
+        Arrays.asList(new Tuple2<String, Integer>("california", 14),
+            new Tuple2<String, Integer>("new york", 9)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey(
+        new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
+        @Override
+        public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
+          int out = 0;
+          if (state.isPresent()) {
+            out = out + state.get();
+          }
+          for (Integer v: values) {
+            out = out + v;
+          }
+          return Optional.of(out);
+        }
+        });
+    JavaTestUtils.attachTestOutputStream(updated);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testReduceByKeyAndWindowWithInverse() {
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
+
+    List<List<Tuple2<String, Integer>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<String, Integer>("california", 4),
+            new Tuple2<String, Integer>("new york", 5)),
+        Arrays.asList(new Tuple2<String, Integer>("california", 14),
+            new Tuple2<String, Integer>("new york", 9)),
+        Arrays.asList(new Tuple2<String, Integer>("california", 10),
+            new Tuple2<String, Integer>("new york", 4)));
+
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, Integer> reduceWindowed =
+        pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(reduceWindowed);
+    List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCountByValueAndWindow() {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("hello", "world"),
+        Arrays.asList("hello", "moon"),
+        Arrays.asList("hello"));
+
+    List<List<Tuple2<String, Long>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Long>("hello", 1L),
+            new Tuple2<String, Long>("world", 1L)),
+        Arrays.asList(
+            new Tuple2<String, Long>("hello", 2L),
+            new Tuple2<String, Long>("world", 1L),
+            new Tuple2<String, Long>("moon", 1L)),
+        Arrays.asList(
+            new Tuple2<String, Long>("hello", 2L),
+            new Tuple2<String, Long>("moon", 1L)));
+
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, Long> counted =
+      stream.countByValueAndWindow(new Duration(2000), new Duration(1000));
+    JavaTestUtils.attachTestOutputStream(counted);
+    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairTransform() {
+    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<Integer, Integer>(3, 5),
+            new Tuple2<Integer, Integer>(1, 5),
+            new Tuple2<Integer, Integer>(4, 5),
+            new Tuple2<Integer, Integer>(2, 5)),
+        Arrays.asList(
+            new Tuple2<Integer, Integer>(2, 5),
+            new Tuple2<Integer, Integer>(3, 5),
+            new Tuple2<Integer, Integer>(4, 5),
+            new Tuple2<Integer, Integer>(1, 5)));
+
+    List<List<Tuple2<Integer, Integer>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<Integer, Integer>(1, 5),
+            new Tuple2<Integer, Integer>(2, 5),
+            new Tuple2<Integer, Integer>(3, 5),
+            new Tuple2<Integer, Integer>(4, 5)),
+        Arrays.asList(
+            new Tuple2<Integer, Integer>(1, 5),
+            new Tuple2<Integer, Integer>(2, 5),
+            new Tuple2<Integer, Integer>(3, 5),
+            new Tuple2<Integer, Integer>(4, 5)));
+
+    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<Integer, Integer> sorted = pairStream.transform(
+        new Function<JavaPairRDD<Integer, Integer>, JavaPairRDD<Integer, Integer>>() {
+          @Override
+          public JavaPairRDD<Integer, Integer> call(JavaPairRDD<Integer, Integer> in) throws Exception {
+            return in.sortByKey();
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(sorted);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testPairToNormalRDDTransform() {
+    List<List<Tuple2<Integer, Integer>>> inputData = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<Integer, Integer>(3, 5),
+            new Tuple2<Integer, Integer>(1, 5),
+            new Tuple2<Integer, Integer>(4, 5),
+            new Tuple2<Integer, Integer>(2, 5)),
+        Arrays.asList(
+            new Tuple2<Integer, Integer>(2, 5),
+            new Tuple2<Integer, Integer>(3, 5),
+            new Tuple2<Integer, Integer>(4, 5),
+            new Tuple2<Integer, Integer>(1, 5)));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(3,1,4,2),
+        Arrays.asList(2,3,4,1));
+
+    JavaDStream<Tuple2<Integer, Integer>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaDStream<Integer> firstParts = pairStream.transform(
+        new Function<JavaPairRDD<Integer, Integer>, JavaRDD<Integer>>() {
+          @Override
+          public JavaRDD<Integer> call(JavaPairRDD<Integer, Integer> in) throws Exception {
+            return in.map(new Function<Tuple2<Integer, Integer>, Integer>() {
+              @Override
+              public Integer call(Tuple2<Integer, Integer> in) {
+                return in._1();
+              }
+            });
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(firstParts);
+    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  public void testMapValues() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, String>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "DODGERS"),
+            new Tuple2<String, String>("california", "GIANTS"),
+            new Tuple2<String, String>("new york", "YANKEES"),
+            new Tuple2<String, String>("new york", "METS")),
+        Arrays.asList(new Tuple2<String, String>("california", "SHARKS"),
+            new Tuple2<String, String>("california", "DUCKS"),
+            new Tuple2<String, String>("new york", "RANGERS"),
+            new Tuple2<String, String>("new york", "ISLANDERS")));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+    JavaPairDStream<String, String> mapped = pairStream.mapValues(new Function<String, String>() {
+      @Override
+      public String call(String s) throws Exception {
+        return s.toUpperCase();
+      }
+    });
+
+    JavaTestUtils.attachTestOutputStream(mapped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testFlatMapValues() {
+    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+
+    List<List<Tuple2<String, String>>> expected = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "dodgers1"),
+            new Tuple2<String, String>("california", "dodgers2"),
+            new Tuple2<String, String>("california", "giants1"),
+            new Tuple2<String, String>("california", "giants2"),
+            new Tuple2<String, String>("new york", "yankees1"),
+            new Tuple2<String, String>("new york", "yankees2"),
+            new Tuple2<String, String>("new york", "mets1"),
+            new Tuple2<String, String>("new york", "mets2")),
+        Arrays.asList(new Tuple2<String, String>("california", "sharks1"),
+            new Tuple2<String, String>("california", "sharks2"),
+            new Tuple2<String, String>("california", "ducks1"),
+            new Tuple2<String, String>("california", "ducks2"),
+            new Tuple2<String, String>("new york", "rangers1"),
+            new Tuple2<String, String>("new york", "rangers2"),
+            new Tuple2<String, String>("new york", "islanders1"),
+            new Tuple2<String, String>("new york", "islanders2")));
+
+    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+        ssc, inputData, 1);
+    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+
+
+    JavaPairDStream<String, String> flatMapped = pairStream.flatMapValues(
+        new Function<String, Iterable<String>>() {
+          @Override
+          public Iterable<String> call(String in) {
+            List<String> out = new ArrayList<String>();
+            out.add(in + "1");
+            out.add(in + "2");
+            return out;
+          }
+        });
+
+    JavaTestUtils.attachTestOutputStream(flatMapped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCoGroup() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+            new Tuple2<String, String>("new york", "yankees")),
+        Arrays.asList(new Tuple2<String, String>("california", "sharks"),
+            new Tuple2<String, String>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "giants"),
+            new Tuple2<String, String>("new york", "mets")),
+        Arrays.asList(new Tuple2<String, String>("california", "ducks"),
+            new Tuple2<String, String>("new york", "islanders")));
+
+
+    List<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Tuple2<List<String>, List<String>>>("california",
+                new Tuple2<List<String>, List<String>>(Arrays.asList("dodgers"), Arrays.asList("giants"))),
+            new Tuple2<String, Tuple2<List<String>, List<String>>>("new york",
+                new Tuple2<List<String>, List<String>>(Arrays.asList("yankees"), Arrays.asList("mets")))),
+        Arrays.asList(
+            new Tuple2<String, Tuple2<List<String>, List<String>>>("california",
+                new Tuple2<List<String>, List<String>>(Arrays.asList("sharks"), Arrays.asList("ducks"))),
+            new Tuple2<String, Tuple2<List<String>, List<String>>>("new york",
+                new Tuple2<List<String>, List<String>>(Arrays.asList("rangers"), Arrays.asList("islanders")))));
+
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<List<String>, List<String>>> grouped = pairStream1.cogroup(pairStream2);
+    JavaTestUtils.attachTestOutputStream(grouped);
+    List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testJoin() {
+    List<List<Tuple2<String, String>>> stringStringKVStream1 = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "dodgers"),
+            new Tuple2<String, String>("new york", "yankees")),
+        Arrays.asList(new Tuple2<String, String>("california", "sharks"),
+            new Tuple2<String, String>("new york", "rangers")));
+
+    List<List<Tuple2<String, String>>> stringStringKVStream2 = Arrays.asList(
+        Arrays.asList(new Tuple2<String, String>("california", "giants"),
+            new Tuple2<String, String>("new york", "mets")),
+        Arrays.asList(new Tuple2<String, String>("california", "ducks"),
+            new Tuple2<String, String>("new york", "islanders")));
+
+
+    List<List<Tuple2<String, Tuple2<String, String>>>> expected = Arrays.asList(
+        Arrays.asList(
+            new Tuple2<String, Tuple2<String, String>>("california",
+                new Tuple2<String, String>("dodgers", "giants")),
+            new Tuple2<String, Tuple2<String, String>>("new york",
+                new Tuple2<String, String>("yankees", "mets"))),
+        Arrays.asList(
+            new Tuple2<String, Tuple2<String, String>>("california",
+                new Tuple2<String, String>("sharks", "ducks")),
+            new Tuple2<String, Tuple2<String, String>>("new york",
+                new Tuple2<String, String>("rangers", "islanders"))));
+
+
+    JavaDStream<Tuple2<String, String>> stream1 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream1, 1);
+    JavaPairDStream<String, String> pairStream1 = JavaPairDStream.fromJavaDStream(stream1);
+
+    JavaDStream<Tuple2<String, String>> stream2 = JavaTestUtils.attachTestInputStream(
+        ssc, stringStringKVStream2, 1);
+    JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2);
+
+    JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.join(pairStream2);
+    JavaTestUtils.attachTestOutputStream(joined);
+    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
+
+    Assert.assertEquals(expected, result);
+  }
+
+  @Test
+  public void testCheckpointMasterRecovery() throws InterruptedException {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("this", "is"),
+        Arrays.asList("a", "test"),
+        Arrays.asList("counting", "letters"));
+
+    List<List<Integer>> expectedInitial = Arrays.asList(
+        Arrays.asList(4,2));
+    List<List<Integer>> expectedFinal = Arrays.asList(
+        Arrays.asList(1,4),
+        Arrays.asList(8,7));
+
+    File tempDir = Files.createTempDir();
+    ssc.checkpoint(tempDir.getAbsolutePath());
+
+    JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+      @Override
+      public Integer call(String s) throws Exception {
+        return s.length();
+      }
+    });
+    JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
+    List<List<Integer>> initialResult = JavaTestUtils.runStreams(ssc, 1, 1);
+
+    assertOrderInvariantEquals(expectedInitial, initialResult);
+    Thread.sleep(1000);
+    ssc.stop();
+
+    ssc = new JavaStreamingContext(tempDir.getAbsolutePath());
+    // Tweak to take into consideration that the last batch before failure
+    // will be re-processed after recovery
+    List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 3);
+    assertOrderInvariantEquals(expectedFinal, finalResult.subList(1, 3));
+  }
+
+
+  /** TEST DISABLED: Pending a discussion about checkpoint() semantics with TD
+  @Test
+  public void testCheckpointofIndividualStream() throws InterruptedException {
+    List<List<String>> inputData = Arrays.asList(
+        Arrays.asList("this", "is"),
+        Arrays.asList("a", "test"),
+        Arrays.asList("counting", "letters"));
+
+    List<List<Integer>> expected = Arrays.asList(
+        Arrays.asList(4,2),
+        Arrays.asList(1,4),
+        Arrays.asList(8,7));
+
+    JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+      @Override
+      public Integer call(String s) throws Exception {
+        return s.length();
+      }
+    });
+    JavaCheckpointTestUtils.attachTestOutputStream(letterCount);
+
+    letterCount.checkpoint(new Duration(1000));
+
+    List<List<Integer>> result1 = JavaCheckpointTestUtils.runStreams(ssc, 3, 3);
+    assertOrderInvariantEquals(expected, result1);
+  }
+  */
+
+  // Input stream tests. These mostly just test that we can instantiate a given InputStream with
+  // Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
+  // InputStream functionality is deferred to the existing Scala tests.
+  @Test
+  public void testKafkaStream() {
+    HashMap<String, Integer> topics = Maps.newHashMap();
+    HashMap<KafkaPartitionKey, Long> offsets = Maps.newHashMap();
+    JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics);
+    JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, offsets);
+    JavaDStream test3 = ssc.kafkaStream("localhost:12345", "group", topics, offsets,
+      StorageLevel.MEMORY_AND_DISK());
+  }
+
+  @Test
+  public void testSocketTextStream() {
+    JavaDStream test = ssc.socketTextStream("localhost", 12345);
+  }
+
+  @Test
+  public void testSocketString() {
+    class Converter extends Function<InputStream, Iterable<String>> {
+      public Iterable<String> call(InputStream in) {
+        BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+        List<String> out = new ArrayList<String>();
+        try {
+          while (true) {
+            String line = reader.readLine();
+            if (line == null) { break; }
+            out.add(line);
+          }
+        } catch (IOException e) { }
+        return out;
+      }
+    }
+
+    JavaDStream test = ssc.socketStream(
+      "localhost",
+      12345,
+      new Converter(),
+      StorageLevel.MEMORY_ONLY());
+  }
+
+  @Test
+  public void testTextFileStream() {
+    JavaDStream test = ssc.textFileStream("/tmp/foo");
+  }
+
+  @Test
+  public void testRawSocketStream() {
+    JavaDStream test = ssc.rawSocketStream("localhost", 12345);
+  }
+
+  @Test
+  public void testFlumeStream() {
+    JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
+  }
+
+  @Test
+  public void testFileStream() {
+    JavaPairDStream<String, String> foo =
+      ssc.<String, String, SequenceFileInputFormat>fileStream("/tmp/foo");
+  }
+
+  @Test
+  public void testTwitterStream() {
+    String[] filters = new String[] { "good", "bad", "ugly" };
+    JavaDStream test = ssc.twitterStream("username", "password", filters, StorageLevel.MEMORY_ONLY());
+  }
+
+  @Test
+  public void testActorStream() {
+    JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
+  }
+
+  @Test
+  public void testZeroMQStream() {
+    JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
+      @Override
+      public Iterable<String> call(byte[][] b) throws Exception {
+        return null;
+      }
+    });
+  }
+}
diff --git a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
new file mode 100644
index 0000000000000000000000000000000000000000..64a7e7cbf9a367285b9a3704f781235dac099bbb
--- /dev/null
+++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
@@ -0,0 +1,67 @@
+package spark.streaming
+
+import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import java.util.{List => JList}
+import spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext}
+import spark.streaming._
+import java.util.ArrayList
+import collection.JavaConversions._
+
+/** Exposes streaming test functionality in a Java-friendly way. */
+trait JavaTestBase extends TestSuiteBase {
+
+  /**
+   * Create a [[spark.streaming.TestInputStream]] and attach it to the supplied context.
+   * The stream will be derived from the supplied lists of Java objects.
+   **/
+  def attachTestInputStream[T](
+    ssc: JavaStreamingContext,
+    data: JList[JList[T]],
+    numPartitions: Int) = {
+    val seqData = data.map(Seq(_:_*))
+
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    val dstream = new TestInputStream[T](ssc.ssc, seqData, numPartitions)
+    ssc.ssc.registerInputStream(dstream)
+    new JavaDStream[T](dstream)
+  }
+
+  /**
+   * Attach a provided stream to it's associated StreamingContext as a
+   * [[spark.streaming.TestOutputStream]].
+   **/
+  def attachTestOutputStream[T, This <: spark.streaming.api.java.JavaDStreamLike[T, This, R],
+      R <: spark.api.java.JavaRDDLike[T, R]](
+    dstream: JavaDStreamLike[T, This, R]) = {
+    implicit val cm: ClassManifest[T] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+    val ostream = new TestOutputStream(dstream.dstream,
+      new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]])
+    dstream.dstream.ssc.registerOutputStream(ostream)
+  }
+
+  /**
+   * Process all registered streams for a numBatches batches, failing if
+   * numExpectedOutput RDD's are not generated. Generated RDD's are collected
+   * and returned, represented as a list for each batch interval.
+   */
+  def runStreams[V](
+    ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = {
+    implicit val cm: ClassManifest[V] =
+      implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+    val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput)
+    val out = new ArrayList[JList[V]]()
+    res.map(entry => out.append(new ArrayList[V](entry)))
+    out
+  }
+}
+
+object JavaTestUtils extends JavaTestBase {
+  override def maxWaitTimeMillis = 20000
+
+}
+
+object JavaCheckpointTestUtils extends JavaTestBase {
+  override def actuallyWait = true
+}
\ No newline at end of file
diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000000000000000000000000000000..59c445e63f7974163e77d47257644474dd14e62f
--- /dev/null
+++ b/streaming/src/test/resources/log4j.properties
@@ -0,0 +1,12 @@
+# Set everything to be logged to the file streaming/target/unit-tests.log 
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..cf2ed8b1d4bef11018e27785702cb44254527188
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -0,0 +1,304 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import scala.runtime.RichInt
+import util.ManualClock
+
+class BasicOperationsSuite extends TestSuiteBase {
+
+  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")
+  }
+
+  test("map") {
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    testOperation(
+      input,
+      (r: DStream[Int]) => r.map(_.toString),
+      input.map(_.map(_.toString))
+    )
+  }
+
+  test("flatMap") {
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    testOperation(
+      input,
+      (r: DStream[Int]) => r.flatMap(x => Seq(x, x * 2)),
+      input.map(_.flatMap(x => Array(x, x * 2)))
+    )
+  }
+
+  test("filter") {
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    testOperation(
+      input,
+      (r: DStream[Int]) => r.filter(x => (x % 2 == 0)),
+      input.map(_.filter(x => (x % 2 == 0)))
+    )
+  }
+
+  test("glom") {
+    assert(numInputPartitions === 2, "Number of input partitions has been changed from 2")
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    val output = Seq(
+      Seq( Seq(1, 2), Seq(3, 4) ),
+      Seq( Seq(5, 6), Seq(7, 8) ),
+      Seq( Seq(9, 10), Seq(11, 12) )
+    )
+    val operation = (r: DStream[Int]) => r.glom().map(_.toSeq)
+    testOperation(input, operation, output)
+  }
+
+  test("mapPartitions") {
+    assert(numInputPartitions === 2, "Number of input partitions has been changed from 2")
+    val input = Seq(1 to 4, 5 to 8, 9 to 12)
+    val output = Seq(Seq(3, 7), Seq(11, 15), Seq(19, 23))
+    val operation = (r: DStream[Int]) => r.mapPartitions(x => Iterator(x.reduce(_ + _)))
+    testOperation(input, operation, output, true)
+  }
+
+  test("groupByKey") {
+    testOperation(
+      Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+      (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(),
+      Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ),
+      true
+    )
+  }
+
+  test("reduceByKey") {
+    testOperation(
+      Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+      (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _),
+      Seq( Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq() ),
+      true
+    )
+  }
+
+  test("reduce") {
+    testOperation(
+      Seq(1 to 4, 5 to 8, 9 to 12),
+      (s: DStream[Int]) => s.reduce(_ + _),
+      Seq(Seq(10), Seq(26), Seq(42))
+    )
+  }
+
+  test("count") {
+    testOperation(
+      Seq(1 to 1, 1 to 2, 1 to 3, 1 to 4),
+      (s: DStream[Int]) => s.count(),
+      Seq(Seq(1L), Seq(2L), Seq(3L), Seq(4L))
+    )
+  }
+
+  test("countByValue") {
+    testOperation(
+      Seq(1 to 1, Seq(1, 1, 1), 1 to 2, Seq(1, 1, 2, 2)),
+      (s: DStream[Int]) => s.countByValue(),
+      Seq(Seq((1, 1L)), Seq((1, 3L)), Seq((1, 1L), (2, 1L)), Seq((2, 2L), (1, 2L))),
+      true
+    )
+  }
+
+  test("mapValues") {
+    testOperation(
+      Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+      (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _).mapValues(_ + 10),
+      Seq( Seq(("a", 12), ("b", 11)), Seq(("", 12)), Seq() ),
+      true
+    )
+  }
+
+  test("flatMapValues") {
+    testOperation(
+      Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ),
+      (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _).flatMapValues(x => Seq(x, x + 10)),
+      Seq( Seq(("a", 2), ("a", 12), ("b", 1), ("b", 11)), Seq(("", 2), ("", 12)), Seq() ),
+      true
+    )
+  }
+
+  test("cogroup") {
+    val inputData1 = Seq( Seq("a", "a", "b"), Seq("a", ""), Seq(""), Seq() )
+    val inputData2 = Seq( Seq("a", "a", "b"), Seq("b", ""), Seq(), Seq()   )
+    val outputData = Seq(
+      Seq( ("a", (Seq(1, 1), Seq("x", "x"))), ("b", (Seq(1), Seq("x"))) ),
+      Seq( ("a", (Seq(1), Seq())), ("b", (Seq(), Seq("x"))), ("", (Seq(1), Seq("x"))) ),
+      Seq( ("", (Seq(1), Seq())) ),
+      Seq(  )
+    )
+    val operation = (s1: DStream[String], s2: DStream[String]) => {
+      s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x")))
+    }
+    testOperation(inputData1, inputData2, operation, outputData, true)
+  }
+
+  test("join") {
+    val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() )
+    val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("")   )
+    val outputData = Seq(
+      Seq( ("a", (1, "x")), ("b", (1, "x")) ),
+      Seq( ("", (1, "x")) ),
+      Seq(  ),
+      Seq(  )
+    )
+    val operation = (s1: DStream[String], s2: DStream[String]) => {
+      s1.map(x => (x,1)).join(s2.map(x => (x,"x")))
+    }
+    testOperation(inputData1, inputData2, operation, outputData, true)
+  }
+
+  test("updateStateByKey") {
+    val inputData =
+      Seq(
+        Seq("a"),
+        Seq("a", "b"),
+        Seq("a", "b", "c"),
+        Seq("a", "b"),
+        Seq("a"),
+        Seq()
+      )
+
+    val outputData =
+      Seq(
+        Seq(("a", 1)),
+        Seq(("a", 2), ("b", 1)),
+        Seq(("a", 3), ("b", 2), ("c", 1)),
+        Seq(("a", 4), ("b", 3), ("c", 1)),
+        Seq(("a", 5), ("b", 3), ("c", 1)),
+        Seq(("a", 5), ("b", 3), ("c", 1))
+      )
+
+    val updateStateOperation = (s: DStream[String]) => {
+      val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+        Some(values.foldLeft(0)(_ + _) + state.getOrElse(0))
+      }
+      s.map(x => (x, 1)).updateStateByKey[Int](updateFunc)
+    }
+
+    testOperation(inputData, updateStateOperation, outputData, true)
+  }
+
+  test("updateStateByKey - object lifecycle") {
+    val inputData =
+      Seq(
+        Seq("a","b"),
+        null,
+        Seq("a","c","a"),
+        Seq("c"),
+        null,
+        null
+      )
+
+    val outputData =
+      Seq(
+        Seq(("a", 1), ("b", 1)),
+        Seq(("a", 1), ("b", 1)),
+        Seq(("a", 3), ("c", 1)),
+        Seq(("a", 3), ("c", 2)),
+        Seq(("c", 2)),
+        Seq()
+      )
+
+    val updateStateOperation = (s: DStream[String]) => {
+      class StateObject(var counter: Int = 0, var expireCounter: Int = 0) extends Serializable
+
+      // updateFunc clears a state when a StateObject is seen without new values twice in a row
+      val updateFunc = (values: Seq[Int], state: Option[StateObject]) => {
+        val stateObj = state.getOrElse(new StateObject)
+        values.foldLeft(0)(_ + _) match {
+          case 0 => stateObj.expireCounter += 1 // no new values
+          case n => { // has new values, increment and reset expireCounter
+            stateObj.counter += n
+            stateObj.expireCounter = 0
+          }
+        }
+        stateObj.expireCounter match {
+          case 2 => None // seen twice with no new values, give it the boot
+          case _ => Option(stateObj)
+        }
+      }
+      s.map(x => (x, 1)).updateStateByKey[StateObject](updateFunc).mapValues(_.counter)
+    }
+
+    testOperation(inputData, updateStateOperation, outputData, true)
+  }
+
+  test("slice") {
+    val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1))
+    val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
+    val stream = new TestInputStream[Int](ssc, input, 2)
+    ssc.registerInputStream(stream)
+    stream.foreach(_ => {})  // Dummy output stream
+    ssc.start()
+    Thread.sleep(2000)
+    def getInputFromSlice(fromMillis: Long, toMillis: Long) = {
+      stream.slice(new Time(fromMillis), new Time(toMillis)).flatMap(_.collect()).toSet
+    }
+
+    assert(getInputFromSlice(0, 1000) == Set(1))
+    assert(getInputFromSlice(0, 2000) == Set(1, 2))
+    assert(getInputFromSlice(1000, 2000) == Set(1, 2))
+    assert(getInputFromSlice(2000, 4000) == Set(2, 3, 4))
+    ssc.stop()
+    Thread.sleep(1000)
+  }
+
+  test("forgetting of RDDs - map and window operations") {
+    assert(batchDuration === Seconds(1), "Batch duration has changed from 1 second")
+
+    val input = (0 until 10).map(x => Seq(x, x + 1)).toSeq
+    val rememberDuration = Seconds(3)
+
+    assert(input.size === 10, "Number of inputs have changed")
+
+    def operation(s: DStream[Int]): DStream[(Int, Int)] = {
+      s.map(x => (x % 10, 1))
+       .window(Seconds(2), Seconds(1))
+       .window(Seconds(4), Seconds(2))
+    }
+
+    val ssc = setupStreams(input, operation _)
+    ssc.remember(rememberDuration)
+    runStreams[(Int, Int)](ssc, input.size, input.size / 2)
+
+    val windowedStream2 = ssc.graph.getOutputStreams().head.dependencies.head
+    val windowedStream1 = windowedStream2.dependencies.head
+    val mappedStream = windowedStream1.dependencies.head
+
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    assert(clock.time === Seconds(10).milliseconds)
+
+    // IDEALLY
+    // WindowedStream2 should remember till 7 seconds: 10, 8,
+    // WindowedStream1 should remember till 4 seconds: 10, 9, 8, 7, 6, 5
+    // MappedStream should remember till 7 seconds:    10, 9, 8, 7, 6, 5, 4, 3,
+
+    // IN THIS TEST
+    // WindowedStream2 should remember till 7 seconds: 10, 8,
+    // WindowedStream1 should remember till 4 seconds: 10, 9, 8, 7, 6, 5, 4
+    // MappedStream should remember till 7 seconds:    10, 9, 8, 7, 6, 5, 4, 3, 2
+
+    // WindowedStream2
+    assert(windowedStream2.generatedRDDs.contains(Time(10000)))
+    assert(windowedStream2.generatedRDDs.contains(Time(8000)))
+    assert(!windowedStream2.generatedRDDs.contains(Time(6000)))
+
+    // WindowedStream1
+    assert(windowedStream1.generatedRDDs.contains(Time(10000)))
+    assert(windowedStream1.generatedRDDs.contains(Time(4000)))
+    assert(!windowedStream1.generatedRDDs.contains(Time(3000)))
+
+    // MappedStream
+    assert(mappedStream.generatedRDDs.contains(Time(10000)))
+    assert(mappedStream.generatedRDDs.contains(Time(2000)))
+    assert(!mappedStream.generatedRDDs.contains(Time(1000)))
+  }
+}
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..cac86deeaf3492cb298c26d9812cc82e71fd83cf
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
@@ -0,0 +1,353 @@
+package spark.streaming
+
+import dstream.FileInputDStream
+import spark.streaming.StreamingContext._
+import java.io.File
+import runtime.RichInt
+import org.scalatest.BeforeAndAfter
+import org.apache.commons.io.FileUtils
+import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import util.{Clock, ManualClock}
+import scala.util.Random
+import com.google.common.io.Files
+
+
+/**
+ * This test suites tests the checkpointing functionality of DStreams -
+ * the checkpointing of a DStream's RDDs as well as the checkpointing of
+ * the whole DStream graph.
+ */
+class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
+
+  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
+  before {
+    FileUtils.deleteDirectory(new File(checkpointDir))
+  }
+
+  after {
+    if (ssc != null) ssc.stop()
+    FileUtils.deleteDirectory(new File(checkpointDir))
+
+    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+    System.clearProperty("spark.driver.port")
+  }
+
+  var ssc: StreamingContext = null
+
+  override def framework = "CheckpointSuite"
+
+  override def batchDuration = Milliseconds(500)
+
+  override def actuallyWait = true
+
+  test("basic rdd checkpoints + dstream graph checkpoint recovery") {
+
+    assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
+
+    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
+    val stateStreamCheckpointInterval = Seconds(1)
+
+    // this ensure checkpointing occurs at least once
+    val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2
+    val secondNumBatches = firstNumBatches
+
+    // Setup the streams
+    val input = (1 to 10).map(_ => Seq("a")).toSeq
+    val operation = (st: DStream[String]) => {
+      val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
+        Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+      }
+      st.map(x => (x, 1))
+      .updateStateByKey[RichInt](updateFunc)
+      .checkpoint(stateStreamCheckpointInterval)
+      .map(t => (t._1, t._2.self))
+    }
+    var ssc = setupStreams(input, operation)
+    var stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
+
+    // Run till a time such that at least one RDD in the stream should have been checkpointed,
+    // then check whether some RDD has been checkpointed or not
+    ssc.start()
+    advanceTimeWithRealDelay(ssc, firstNumBatches)
+    logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData)
+    assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure")
+    stateStream.checkpointData.checkpointFiles.foreach {
+      case (time, data) => {
+        val file = new File(data.toString)
+        assert(file.exists(), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist")
+      }
+    }
+
+    // Run till a further time such that previous checkpoint files in the stream would be deleted
+    // and check whether the earlier checkpoint files are deleted
+    val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2))
+    advanceTimeWithRealDelay(ssc, secondNumBatches)
+    checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
+    ssc.stop()
+
+    // Restart stream computation using the checkpoint file and check whether
+    // checkpointed RDDs have been restored or not
+    ssc = new StreamingContext(checkpointDir)
+    stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
+    logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
+    assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from first failure")
+
+
+    // Run one batch to generate a new checkpoint file and check whether some RDD
+    // is present in the checkpoint data or not
+    ssc.start()
+    advanceTimeWithRealDelay(ssc, 1)
+    assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure")
+    stateStream.checkpointData.checkpointFiles.foreach {
+      case (time, data) => {
+        val file = new File(data.toString)
+        assert(file.exists(),
+          "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist")
+      }
+    }
+    ssc.stop()
+
+    // Restart stream computation from the new checkpoint file to see whether that file has
+    // correct checkpoint data
+    ssc = new StreamingContext(checkpointDir)
+    stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
+    logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
+    assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure")
+
+    // Adjust manual clock time as if it is being restarted after a delay
+    System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
+    ssc.start()
+    advanceTimeWithRealDelay(ssc, 4)
+    ssc.stop()
+    System.clearProperty("spark.streaming.manualClock.jump")
+    ssc = null
+  }
+
+  // This tests whether the systm can recover from a master failure with simple
+  // non-stateful operations. This assumes as reliable, replayable input
+  // source - TestInputDStream.
+  test("recovery with map and reduceByKey operations") {
+    testCheckpointedOperation(
+      Seq( Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq() ),
+      (s: DStream[String]) => s.map(x => (x, 1)).reduceByKey(_ + _),
+      Seq( Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq() ),
+      3
+    )
+  }
+
+
+  // This tests whether the ReduceWindowedDStream's RDD checkpoints works correctly such
+  // that the system can recover from a master failure. This assumes as reliable,
+  // replayable input source - TestInputDStream.
+  test("recovery with invertible reduceByKeyAndWindow operation") {
+    val n = 10
+    val w = 4
+    val input = (1 to n).map(_ => Seq("a")).toSeq
+    val output = Seq(Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 3))) ++ (1 to (n - w + 1)).map(x => Seq(("a", 4)))
+    val operation = (st: DStream[String]) => {
+      st.map(x => (x, 1))
+        .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration)
+        .checkpoint(batchDuration * 2)
+    }
+    testCheckpointedOperation(input, operation, output, 7)
+  }
+
+
+  // This tests whether the StateDStream's RDD checkpoints works correctly such
+  // that the system can recover from a master failure. This assumes as reliable,
+  // replayable input source - TestInputDStream.
+  test("recovery with updateStateByKey operation") {
+    val input = (1 to 10).map(_ => Seq("a")).toSeq
+    val output = (1 to 10).map(x => Seq(("a", x))).toSeq
+    val operation = (st: DStream[String]) => {
+      val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
+        Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+      }
+      st.map(x => (x, 1))
+        .updateStateByKey[RichInt](updateFunc)
+        .checkpoint(batchDuration * 2)
+        .map(t => (t._1, t._2.self))
+    }
+    testCheckpointedOperation(input, operation, output, 7)
+  }
+
+  // This tests whether file input stream remembers what files were seen before
+  // the master failure and uses them again to process a large window operation.
+  // It also tests whether batches, whose processing was incomplete due to the
+  // failure, are re-processed or not.
+  test("recovery with file input stream") {
+    // Disable manual clock as FileInputDStream does not work with manual clock
+    val clockProperty = System.getProperty("spark.streaming.clock")
+    System.clearProperty("spark.streaming.clock")
+
+    // Set up the streaming context and input streams
+    val testDir = Files.createTempDir()
+    var ssc = new StreamingContext(master, framework, Seconds(1))
+    ssc.checkpoint(checkpointDir)
+    val fileStream = ssc.textFileStream(testDir.toString)
+    // Making value 3 take large time to process, to ensure that the master
+    // shuts down in the middle of processing the 3rd batch
+    val mappedStream = fileStream.map(s => {
+      val i = s.toInt
+      if (i == 3) Thread.sleep(2000)
+      i
+    })
+
+    // Reducing over a large window to ensure that recovery from master failure
+    // requires reprocessing of all the files seen before the failure
+    val reducedStream = mappedStream.reduceByWindow(_ + _, Seconds(30), Seconds(1))
+    val outputBuffer = new ArrayBuffer[Seq[Int]]
+    var outputStream = new TestOutputStream(reducedStream, outputBuffer)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    // Create files and advance manual clock to process them
+    //var clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    Thread.sleep(1000)
+    for (i <- Seq(1, 2, 3)) {
+      FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
+      // wait to make sure that the file is written such that it gets shown in the file listings
+      Thread.sleep(1000)
+    }
+    logInfo("Output = " + outputStream.output.mkString(","))
+    assert(outputStream.output.size > 0, "No files processed before restart")
+    ssc.stop()
+
+    // Verify whether files created have been recorded correctly or not
+    var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]]
+    def recordedFiles = fileInputDStream.files.values.flatMap(x => x)
+    assert(!recordedFiles.filter(_.endsWith("1")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("2")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("3")).isEmpty)
+
+    // Create files while the master is down
+    for (i <- Seq(4, 5, 6)) {
+      FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
+      Thread.sleep(1000)
+    }
+
+    // Recover context from checkpoint file and verify whether the files that were
+    // recorded before failure were saved and successfully recovered
+    logInfo("*********** RESTARTING ************")
+    ssc = new StreamingContext(checkpointDir)
+    fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]]
+    assert(!recordedFiles.filter(_.endsWith("1")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("2")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("3")).isEmpty)
+
+    // Restart stream computation
+    ssc.start()
+    for (i <- Seq(7, 8, 9)) {
+      FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
+      Thread.sleep(1000)
+    }
+    Thread.sleep(1000)
+    logInfo("Output = " + outputStream.output.mkString("[", ", ", "]"))
+    assert(outputStream.output.size > 0, "No files processed after restart")
+    ssc.stop()
+
+    // Verify whether files created while the driver was down have been recorded or not
+    assert(!recordedFiles.filter(_.endsWith("4")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("5")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("6")).isEmpty)
+
+    // Verify whether new files created after recover have been recorded or not
+    assert(!recordedFiles.filter(_.endsWith("7")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("8")).isEmpty)
+    assert(!recordedFiles.filter(_.endsWith("9")).isEmpty)
+
+    // Append the new output to the old buffer
+    outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]]
+    outputBuffer ++= outputStream.output
+
+    val expectedOutput = Seq(1, 3, 6, 10, 15, 21, 28, 36, 45)
+    logInfo("--------------------------------")
+    logInfo("output, size = " + outputBuffer.size)
+    outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("expected output, size = " + expectedOutput.size)
+    expectedOutput.foreach(x => logInfo("[" + x + "]"))
+    logInfo("--------------------------------")
+
+    // Verify whether all the elements received are as expected
+    val output = outputBuffer.flatMap(x => x)
+    assert(output.contains(6))  // To ensure that the 3rd input (i.e., 3) was processed
+    output.foreach(o =>         // To ensure all the inputs are correctly added cumulatively
+      assert(expectedOutput.contains(o), "Expected value " + o + " not found")
+    )
+    // To ensure that all the inputs were received correctly
+    assert(expectedOutput.last === output.last)
+
+    // Enable manual clock back again for other tests
+    if (clockProperty != null)
+      System.setProperty("spark.streaming.clock", clockProperty)
+  }
+
+
+  /**
+   * Tests a streaming operation under checkpointing, by restarting the operation
+   * from checkpoint file and verifying whether the final output is correct.
+   * The output is assumed to have come from a reliable queue which an replay
+   * data as required.
+   *
+   * NOTE: This takes into consideration that the last batch processed before
+   * master failure will be re-processed after restart/recovery.
+   */
+  def testCheckpointedOperation[U: ClassManifest, V: ClassManifest](
+    input: Seq[Seq[U]],
+    operation: DStream[U] => DStream[V],
+    expectedOutput: Seq[Seq[V]],
+    initialNumBatches: Int
+  ) {
+
+    // Current code assumes that:
+    // number of inputs = number of outputs = number of batches to be run
+    val totalNumBatches = input.size
+    val nextNumBatches = totalNumBatches - initialNumBatches
+    val initialNumExpectedOutputs = initialNumBatches
+    val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs + 1
+    // because the last batch will be processed again
+
+    // Do the computation for initial number of batches, create checkpoint file and quit
+    ssc = setupStreams[U, V](input, operation)
+    ssc.start()
+    val output = advanceTimeWithRealDelay[V](ssc, initialNumBatches)
+    ssc.stop()
+    verifyOutput[V](output, expectedOutput.take(initialNumBatches), true)
+    Thread.sleep(1000)
+
+    // Restart and complete the computation from checkpoint file
+    logInfo(
+      "\n-------------------------------------------\n" +
+      "        Restarting stream computation          " +
+      "\n-------------------------------------------\n"
+    )
+    ssc = new StreamingContext(checkpointDir)
+    System.clearProperty("spark.driver.port")
+    ssc.start()
+    val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches)
+    // the first element will be re-processed data of the last batch before restart
+    verifyOutput[V](outputNew, expectedOutput.takeRight(nextNumExpectedOutputs), true)
+    ssc.stop()
+    ssc = null
+  }
+
+  /**
+   * Advances the manual clock on the streaming scheduler by given number of batches.
+   * It also waits for the expected amount of time for each batch.
+   */
+  def advanceTimeWithRealDelay[V: ClassManifest](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = {
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    logInfo("Manual clock before advancing = " + clock.time)
+    for (i <- 1 to numBatches.toInt) {
+      clock.addToTime(batchDuration.milliseconds)
+      Thread.sleep(batchDuration.milliseconds)
+    }
+    logInfo("Manual clock after advancing = " + clock.time)
+    Thread.sleep(batchDuration.milliseconds)
+
+    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/FailureSuite.scala b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..a5fa7ab92dd6bf9564e1f33aba39d82a51deda04
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
@@ -0,0 +1,40 @@
+package spark.streaming
+
+import spark.Logging
+import spark.streaming.util.MasterFailureTest
+import StreamingContext._
+
+import org.scalatest.{FunSuite, BeforeAndAfter}
+import com.google.common.io.Files
+import java.io.File
+import org.apache.commons.io.FileUtils
+import collection.mutable.ArrayBuffer
+
+
+/**
+ * This testsuite tests master failures at random times while the stream is running using
+ * the real clock.
+ */
+class FailureSuite extends FunSuite with BeforeAndAfter with Logging {
+
+  var directory = "FailureSuite"
+  val numBatches = 30
+  val batchDuration = Milliseconds(1000)
+
+  before {
+    FileUtils.deleteDirectory(new File(directory))
+  }
+
+  after {
+    FileUtils.deleteDirectory(new File(directory))
+  }
+
+  test("multiple failures with map") {
+    MasterFailureTest.testMap(directory, numBatches, batchDuration)
+  }
+
+  test("multiple failures with updateStateByKey") {
+    MasterFailureTest.testUpdateStateByKey(directory, numBatches, batchDuration)
+  }
+}
+
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..67dca2ac3121596d0b012ec0e749880a6db94e90
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -0,0 +1,320 @@
+package spark.streaming
+
+import akka.actor.Actor
+import akka.actor.IO
+import akka.actor.IOManager
+import akka.actor.Props
+import akka.util.ByteString
+
+import dstream.SparkFlumeEvent
+import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
+import java.io.{File, BufferedWriter, OutputStreamWriter}
+import java.util.concurrent.{TimeUnit, ArrayBlockingQueue}
+import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import util.ManualClock
+import spark.storage.StorageLevel
+import spark.streaming.receivers.Receiver
+import spark.Logging
+import scala.util.Random
+import org.apache.commons.io.FileUtils
+import org.scalatest.BeforeAndAfter
+import org.apache.flume.source.avro.AvroSourceProtocol
+import org.apache.flume.source.avro.AvroFlumeEvent
+import org.apache.flume.source.avro.Status
+import org.apache.avro.ipc.{specific, NettyTransceiver}
+import org.apache.avro.ipc.specific.SpecificRequestor
+import java.nio.ByteBuffer
+import collection.JavaConversions._
+import java.nio.charset.Charset
+import com.google.common.io.Files
+
+class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
+
+  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")
+  }
+
+
+  test("socket input stream") {
+    // Start the server
+    val testServer = new TestServer()
+    testServer.start()
+
+    // Set up the streaming context and input streams
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
+    val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String  ]]
+    val outputStream = new TestOutputStream(networkStream, outputBuffer)
+    def output = outputBuffer.flatMap(x => x)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    // Feed data to the server to send to the network receiver
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    val input = Seq(1, 2, 3, 4, 5)
+    val expectedOutput = input.map(_.toString)
+    Thread.sleep(1000)
+    for (i <- 0 until input.size) {
+      testServer.send(input(i).toString + "\n")
+      Thread.sleep(500)
+      clock.addToTime(batchDuration.milliseconds)
+    }
+    Thread.sleep(1000)
+    logInfo("Stopping server")
+    testServer.stop()
+    logInfo("Stopping context")
+    ssc.stop()
+
+    // Verify whether data received was as expected
+    logInfo("--------------------------------")
+    logInfo("output.size = " + outputBuffer.size)
+    logInfo("output")
+    outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("expected output.size = " + expectedOutput.size)
+    logInfo("expected output")
+    expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("--------------------------------")
+
+    // Verify whether all the elements received are as expected
+    // (whether the elements were received one in each interval is not verified)
+    assert(output.size === expectedOutput.size)
+    for (i <- 0 until output.size) {
+      assert(output(i) === expectedOutput(i))
+    }
+  }
+
+
+  test("flume input stream") {
+    // Set up the streaming context and input streams
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+    val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
+      with SynchronizedBuffer[Seq[SparkFlumeEvent]]
+    val outputStream = new TestOutputStream(flumeStream, outputBuffer)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    val input = Seq(1, 2, 3, 4, 5)
+    Thread.sleep(1000)
+    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort));
+    val client = SpecificRequestor.getClient(
+      classOf[AvroSourceProtocol], transceiver);
+
+    for (i <- 0 until input.size) {
+      val event = new AvroFlumeEvent
+      event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
+      event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
+      client.append(event)
+      Thread.sleep(500)
+      clock.addToTime(batchDuration.milliseconds)
+    }
+
+    val startTime = System.currentTimeMillis()
+    while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+      logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
+      Thread.sleep(100)
+    }
+    Thread.sleep(1000)
+    val timeTaken = System.currentTimeMillis() - startTime
+    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+    logInfo("Stopping context")
+    ssc.stop()
+
+    val decoder = Charset.forName("UTF-8").newDecoder()
+
+    assert(outputBuffer.size === input.length)
+    for (i <- 0 until outputBuffer.size) {
+      assert(outputBuffer(i).size === 1)
+      val str = decoder.decode(outputBuffer(i).head.event.getBody)
+      assert(str.toString === input(i).toString)
+      assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
+    }
+  }
+
+
+  test("file input stream") {
+    // Disable manual clock as FileInputDStream does not work with manual clock
+    System.clearProperty("spark.streaming.clock")
+
+    // Set up the streaming context and input streams
+    val testDir = Files.createTempDir()
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val fileStream = ssc.textFileStream(testDir.toString)
+    val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
+    def output = outputBuffer.flatMap(x => x)
+    val outputStream = new TestOutputStream(fileStream, outputBuffer)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    // Create files in the temporary directory so that Spark Streaming can read data from it
+    val input = Seq(1, 2, 3, 4, 5)
+    val expectedOutput = input.map(_.toString)
+    Thread.sleep(1000)
+    for (i <- 0 until input.size) {
+      val file = new File(testDir, i.toString)
+      FileUtils.writeStringToFile(file, input(i).toString + "\n")
+      logInfo("Created file " + file)
+      Thread.sleep(batchDuration.milliseconds)
+      Thread.sleep(1000)
+    }
+    val startTime = System.currentTimeMillis()
+    Thread.sleep(1000)
+    val timeTaken = System.currentTimeMillis() - startTime
+    assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+    logInfo("Stopping context")
+    ssc.stop()
+
+    // Verify whether data received by Spark Streaming was as expected
+    logInfo("--------------------------------")
+    logInfo("output, size = " + outputBuffer.size)
+    outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("expected output, size = " + expectedOutput.size)
+    expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("--------------------------------")
+
+    // Verify whether all the elements received are as expected
+    // (whether the elements were received one in each interval is not verified)
+    assert(output.toList === expectedOutput.toList)
+
+    FileUtils.deleteDirectory(testDir)
+
+    // Enable manual clock back again for other tests
+    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+  }
+
+
+  test("actor input stream") {
+    // Start the server
+    val testServer = new TestServer()
+    val port = testServer.port
+    testServer.start()
+
+    // Set up the streaming context and input streams
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor",
+      StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope
+    val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
+    val outputStream = new TestOutputStream(networkStream, outputBuffer)
+    def output = outputBuffer.flatMap(x => x)
+    ssc.registerOutputStream(outputStream)
+    ssc.start()
+
+    // Feed data to the server to send to the network receiver
+    val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    val input = 1 to 9
+    val expectedOutput = input.map(x => x.toString)
+    Thread.sleep(1000)
+    for (i <- 0 until input.size) {
+      testServer.send(input(i).toString)
+      Thread.sleep(500)
+      clock.addToTime(batchDuration.milliseconds)
+    }
+    Thread.sleep(1000)
+    logInfo("Stopping server")
+    testServer.stop()
+    logInfo("Stopping context")
+    ssc.stop()
+
+    // Verify whether data received was as expected
+    logInfo("--------------------------------")
+    logInfo("output.size = " + outputBuffer.size)
+    logInfo("output")
+    outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("expected output.size = " + expectedOutput.size)
+    logInfo("expected output")
+    expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("--------------------------------")
+
+    // Verify whether all the elements received are as expected
+    // (whether the elements were received one in each interval is not verified)
+    assert(output.size === expectedOutput.size)
+    for (i <- 0 until output.size) {
+      assert(output(i) === expectedOutput(i))
+    }
+  }
+}
+
+
+/** This is server to test the network input stream */
+class TestServer() extends Logging {
+
+  val queue = new ArrayBlockingQueue[String](100)
+
+  val serverSocket = new ServerSocket(0)
+
+  val servingThread = new Thread() {
+    override def run() {
+      try {
+        while(true) {
+          logInfo("Accepting connections on port " + port)
+          val clientSocket = serverSocket.accept()
+          logInfo("New connection")
+          try {
+            clientSocket.setTcpNoDelay(true)
+            val outputStream = new BufferedWriter(new OutputStreamWriter(clientSocket.getOutputStream))
+
+            while(clientSocket.isConnected) {
+              val msg = queue.poll(100, TimeUnit.MILLISECONDS)
+              if (msg != null) {
+                outputStream.write(msg)
+                outputStream.flush()
+                logInfo("Message '" + msg + "' sent")
+              }
+            }
+          } catch {
+            case e: SocketException => logError("TestServer error", e)
+          } finally {
+            logInfo("Connection closed")
+            if (!clientSocket.isClosed) clientSocket.close()
+          }
+        }
+      } catch {
+        case ie: InterruptedException =>
+
+      } finally {
+        serverSocket.close()
+      }
+    }
+  }
+
+  def start() { servingThread.start() }
+
+  def send(msg: String) { queue.add(msg) }
+
+  def stop() { servingThread.interrupt() }
+
+  def port = serverSocket.getLocalPort
+}
+
+object TestServer {
+  def main(args: Array[String]) {
+    val s = new TestServer()
+    s.start()
+    while(true) {
+      Thread.sleep(1000)
+      s.send("hello")
+    }
+  }
+}
+
+class TestActor(port: Int) extends Actor with Receiver {
+
+  def bytesToString(byteString: ByteString) = byteString.utf8String
+
+  override def preStart = IOManager(context.system).connect(new InetSocketAddress(port))
+
+  def receive = {
+    case IO.Read(socket, bytes) =>
+      pushBlock(bytesToString(bytes))
+  }
+}
diff --git a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ad6aa79d102339bc0595d30463d47bdf6ee183fd
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
@@ -0,0 +1,297 @@
+package spark.streaming
+
+import spark.streaming.dstream.{InputDStream, ForEachDStream}
+import spark.streaming.util.ManualClock
+
+import spark.{RDD, Logging}
+
+import collection.mutable.ArrayBuffer
+import collection.mutable.SynchronizedBuffer
+
+import java.io.{ObjectInputStream, IOException}
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+
+/**
+ * This is a input stream just for the testsuites. This is equivalent to a checkpointable,
+ * replayable, reliable message queue like Kafka. It requires a sequence as input, and
+ * returns the i_th element at the i_th batch unde manual clock.
+ */
+class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
+  extends InputDStream[T](ssc_) {
+
+  def start() {}
+
+  def stop() {}
+
+  def compute(validTime: Time): Option[RDD[T]] = {
+    logInfo("Computing RDD for time " + validTime)
+    val index = ((validTime - zeroTime) / slideDuration - 1).toInt
+    val selectedInput = if (index < input.size) input(index) else Seq[T]()
+
+    // lets us test cases where RDDs are not created
+    if (selectedInput == null)
+      return None
+
+    val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
+    logInfo("Created RDD " + rdd.id + " with " + selectedInput)
+    Some(rdd)
+  }
+}
+
+/**
+ * This is a output stream just for the testsuites. All the output is collected into a
+ * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
+ */
+class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBuffer[Seq[T]])
+  extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
+    val collected = rdd.collect()
+    output += collected
+  }) {
+
+  // This is to clear the output buffer every it is read from a checkpoint
+  @throws(classOf[IOException])
+  private def readObject(ois: ObjectInputStream) {
+    ois.defaultReadObject()
+    output.clear()
+  }
+}
+
+/**
+ * This is the base trait for Spark Streaming testsuites. This provides basic functionality
+ * to run user-defined set of input on user-defined stream operations, and verify the output.
+ */
+trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
+
+  // Name of the framework for Spark context
+  def framework = "TestSuiteBase"
+
+  // Master for Spark context
+  def master = "local[2]"
+
+  // Batch duration
+  def batchDuration = Seconds(1)
+
+  // Directory where the checkpoint data will be saved
+  def checkpointDir = "checkpoint"
+
+  // Number of partitions of the input parallel collections created for testing
+  def numInputPartitions = 2
+
+  // Maximum time to wait before the test times out
+  def maxWaitTimeMillis = 10000
+
+  // Whether to actually wait in real time before changing manual clock
+  def actuallyWait = false
+
+  /**
+   * Set up required DStreams to test the DStream operation using the two sequences
+   * of input collections.
+   */
+  def setupStreams[U: ClassManifest, V: ClassManifest](
+      input: Seq[Seq[U]],
+      operation: DStream[U] => DStream[V]
+    ): StreamingContext = {
+
+    // Create StreamingContext
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    if (checkpointDir != null) {
+      ssc.checkpoint(checkpointDir)
+    }
+
+    // Setup the stream computation
+    val inputStream = new TestInputStream(ssc, input, numInputPartitions)
+    val operatedStream = operation(inputStream)
+    val outputStream = new TestOutputStream(operatedStream, new ArrayBuffer[Seq[V]] with SynchronizedBuffer[Seq[V]])
+    ssc.registerInputStream(inputStream)
+    ssc.registerOutputStream(outputStream)
+    ssc
+  }
+
+  /**
+   * Set up required DStreams to test the binary operation using the sequence
+   * of input collections.
+   */
+  def setupStreams[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+      input1: Seq[Seq[U]],
+      input2: Seq[Seq[V]],
+      operation: (DStream[U], DStream[V]) => DStream[W]
+    ): StreamingContext = {
+
+    // Create StreamingContext
+    val ssc = new StreamingContext(master, framework, batchDuration)
+    if (checkpointDir != null) {
+      ssc.checkpoint(checkpointDir)
+    }
+
+    // Setup the stream computation
+    val inputStream1 = new TestInputStream(ssc, input1, numInputPartitions)
+    val inputStream2 = new TestInputStream(ssc, input2, numInputPartitions)
+    val operatedStream = operation(inputStream1, inputStream2)
+    val outputStream = new TestOutputStream(operatedStream, new ArrayBuffer[Seq[W]] with SynchronizedBuffer[Seq[W]])
+    ssc.registerInputStream(inputStream1)
+    ssc.registerInputStream(inputStream2)
+    ssc.registerOutputStream(outputStream)
+    ssc
+  }
+
+  /**
+   * Runs the streams set up in `ssc` on manual clock for `numBatches` batches and
+   * returns the collected output. It will wait until `numExpectedOutput` number of
+   * output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
+   */
+  def runStreams[V: ClassManifest](
+      ssc: StreamingContext,
+      numBatches: Int,
+      numExpectedOutput: Int
+    ): Seq[Seq[V]] = {
+    assert(numBatches > 0, "Number of batches to run stream computation is zero")
+    assert(numExpectedOutput > 0, "Number of expected outputs after " + numBatches + " is zero")
+    logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput)
+
+    // Get the output buffer
+    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
+    val output = outputStream.output
+
+    try {
+      // Start computation
+      ssc.start()
+
+      // Advance manual clock
+      val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+      logInfo("Manual clock before advancing = " + clock.time)
+      if (actuallyWait) {
+        for (i <- 1 to numBatches) {
+          logInfo("Actually waiting for " + batchDuration)
+          clock.addToTime(batchDuration.milliseconds)
+          Thread.sleep(batchDuration.milliseconds)
+        }
+      } else {
+        clock.addToTime(numBatches * batchDuration.milliseconds)
+      }
+      logInfo("Manual clock after advancing = " + clock.time)
+
+      // Wait until expected number of output items have been generated
+      val startTime = System.currentTimeMillis()
+      while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+        logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
+        Thread.sleep(100)
+      }
+      val timeTaken = System.currentTimeMillis() - startTime
+
+      assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+      assert(output.size === numExpectedOutput, "Unexpected number of outputs generated")
+
+      Thread.sleep(500) // Give some time for the forgetting old RDDs to complete
+    } catch {
+      case e: Exception => e.printStackTrace(); throw e;
+    } finally {
+      ssc.stop()
+    }
+    output
+  }
+
+  /**
+   * Verify whether the output values after running a DStream operation
+   * is same as the expected output values, by comparing the output
+   * collections either as lists (order matters) or sets (order does not matter)
+   */
+  def verifyOutput[V: ClassManifest](
+      output: Seq[Seq[V]],
+      expectedOutput: Seq[Seq[V]],
+      useSet: Boolean
+    ) {
+    logInfo("--------------------------------")
+    logInfo("output.size = " + output.size)
+    logInfo("output")
+    output.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("expected output.size = " + expectedOutput.size)
+    logInfo("expected output")
+    expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
+    logInfo("--------------------------------")
+
+    // Match the output with the expected output
+    assert(output.size === expectedOutput.size, "Number of outputs do not match")
+    for (i <- 0 until output.size) {
+      if (useSet) {
+        assert(output(i).toSet === expectedOutput(i).toSet)
+      } else {
+        assert(output(i).toList === expectedOutput(i).toList)
+      }
+    }
+    logInfo("Output verified successfully")
+  }
+
+  /**
+   * Test unary DStream operation with a list of inputs, with number of
+   * batches to run same as the number of expected output values
+   */
+  def testOperation[U: ClassManifest, V: ClassManifest](
+      input: Seq[Seq[U]],
+      operation: DStream[U] => DStream[V],
+      expectedOutput: Seq[Seq[V]],
+      useSet: Boolean = false
+    ) {
+    testOperation[U, V](input, operation, expectedOutput, -1, useSet)
+  }
+
+  /**
+   * Test unary DStream operation with a list of inputs
+   * @param input      Sequence of input collections
+   * @param operation  Binary DStream operation to be applied to the 2 inputs
+   * @param expectedOutput Sequence of expected output collections
+   * @param numBatches Number of batches to run the operation for
+   * @param useSet     Compare the output values with the expected output values
+   *                   as sets (order matters) or as lists (order does not matter)
+   */
+  def testOperation[U: ClassManifest, V: ClassManifest](
+      input: Seq[Seq[U]],
+      operation: DStream[U] => DStream[V],
+      expectedOutput: Seq[Seq[V]],
+      numBatches: Int,
+      useSet: Boolean
+    ) {
+    val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
+    val ssc = setupStreams[U, V](input, operation)
+    val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
+    verifyOutput[V](output, expectedOutput, useSet)
+  }
+
+  /**
+   * Test binary DStream operation with two lists of inputs, with number of
+   * batches to run same as the number of expected output values
+   */
+  def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+      input1: Seq[Seq[U]],
+      input2: Seq[Seq[V]],
+      operation: (DStream[U], DStream[V]) => DStream[W],
+      expectedOutput: Seq[Seq[W]],
+      useSet: Boolean
+    ) {
+    testOperation[U, V, W](input1, input2, operation, expectedOutput, -1, useSet)
+  }
+
+  /**
+   * Test binary DStream operation with two lists of inputs
+   * @param input1     First sequence of input collections
+   * @param input2     Second sequence of input collections
+   * @param operation  Binary DStream operation to be applied to the 2 inputs
+   * @param expectedOutput Sequence of expected output collections
+   * @param numBatches Number of batches to run the operation for
+   * @param useSet     Compare the output values with the expected output values
+   *                   as sets (order matters) or as lists (order does not matter)
+   */
+  def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+      input1: Seq[Seq[U]],
+      input2: Seq[Seq[V]],
+      operation: (DStream[U], DStream[V]) => DStream[W],
+      expectedOutput: Seq[Seq[W]],
+      numBatches: Int,
+      useSet: Boolean
+    ) {
+    val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
+    val ssc = setupStreams[U, V, W](input1, input2, operation)
+    val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
+    verifyOutput[W](output, expectedOutput, useSet)
+  }
+}
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..1b66f3bda20ad0f112295e529f4a3f1419c167ed
--- /dev/null
+++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
@@ -0,0 +1,322 @@
+package spark.streaming
+
+import spark.streaming.StreamingContext._
+import collection.mutable.ArrayBuffer
+
+class WindowOperationsSuite extends TestSuiteBase {
+
+  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
+  override def framework = "WindowOperationsSuite"
+
+  override def maxWaitTimeMillis = 20000
+
+  override def batchDuration = Seconds(1)
+
+  after {
+    // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+    System.clearProperty("spark.driver.port")
+  }
+
+  val largerSlideInput = Seq(
+    Seq(("a", 1)),
+    Seq(("a", 2)),  // 1st window from here
+    Seq(("a", 3)),
+    Seq(("a", 4)),  // 2nd window from here
+    Seq(("a", 5)),
+    Seq(("a", 6)),  // 3rd window from here
+    Seq(),
+    Seq()           // 4th window from here
+  )
+
+  val largerSlideReduceOutput = Seq(
+    Seq(("a", 3)),
+    Seq(("a", 10)),
+    Seq(("a", 18)),
+    Seq(("a", 11))
+  )
+
+
+  val bigInput = Seq(
+    Seq(("a", 1)),
+    Seq(("a", 1), ("b", 1)),
+    Seq(("a", 1), ("b", 1), ("c", 1)),
+    Seq(("a", 1), ("b", 1)),
+    Seq(("a", 1)),
+    Seq(),
+    Seq(("a", 1)),
+    Seq(("a", 1), ("b", 1)),
+    Seq(("a", 1), ("b", 1), ("c", 1)),
+    Seq(("a", 1), ("b", 1)),
+    Seq(("a", 1)),
+    Seq()
+  )
+
+  val bigGroupByOutput = Seq(
+    Seq(("a", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+    Seq(("a", Seq(1))),
+    Seq(("a", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1, 1)), ("c", Seq(1))),
+    Seq(("a", Seq(1, 1)), ("b", Seq(1))),
+    Seq(("a", Seq(1)))
+  )
+
+
+  val bigReduceOutput = Seq(
+    Seq(("a", 1)),
+    Seq(("a", 2), ("b", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 1)),
+    Seq(("a", 1)),
+    Seq(("a", 1)),
+    Seq(("a", 2), ("b", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 1)),
+    Seq(("a", 1))
+  )
+
+  /*
+  The output of the reduceByKeyAndWindow with inverse function but without a filter
+  function will be different from the naive reduceByKeyAndWindow, as no keys get
+  eliminated from the ReducedWindowedDStream even if the value of a key becomes 0.
+  */
+
+  val bigReduceInvOutput = Seq(
+    Seq(("a", 1)),
+    Seq(("a", 2), ("b", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 1), ("c", 0)),
+    Seq(("a", 1), ("b", 0), ("c", 0)),
+    Seq(("a", 1), ("b", 0), ("c", 0)),
+    Seq(("a", 2), ("b", 1), ("c", 0)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 2), ("c", 1)),
+    Seq(("a", 2), ("b", 1), ("c", 0)),
+    Seq(("a", 1), ("b", 0), ("c", 0))
+  )
+
+  // Testing window operation
+
+  testWindow(
+    "basic window",
+    Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+    Seq( Seq(0), Seq(0, 1), Seq(1, 2), Seq(2, 3), Seq(3, 4), Seq(4, 5))
+  )
+
+  testWindow(
+    "tumbling window",
+    Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+    Seq( Seq(0, 1), Seq(2, 3), Seq(4, 5)),
+    Seconds(2),
+    Seconds(2)
+  )
+
+  testWindow(
+    "larger window",
+    Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+    Seq( Seq(0, 1), Seq(0, 1, 2, 3), Seq(2, 3, 4, 5), Seq(4, 5)),
+    Seconds(4),
+    Seconds(2)
+  )
+
+  testWindow(
+    "non-overlapping window",
+    Seq( Seq(0), Seq(1), Seq(2), Seq(3), Seq(4), Seq(5)),
+    Seq( Seq(1, 2), Seq(4, 5)),
+    Seconds(2),
+    Seconds(3)
+  )
+
+  // Testing naive reduceByKeyAndWindow (without invertible function)
+
+  testReduceByKeyAndWindow(
+    "basic reduction",
+    Seq( Seq(("a", 1), ("a", 3)) ),
+    Seq( Seq(("a", 4)) )
+  )
+
+  testReduceByKeyAndWindow(
+    "key already in window and new value added into window",
+    Seq( Seq(("a", 1)), Seq(("a", 1)) ),
+    Seq( Seq(("a", 1)), Seq(("a", 2)) )
+  )
+
+  testReduceByKeyAndWindow(
+    "new key added into window",
+    Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ),
+    Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) )
+  )
+
+  testReduceByKeyAndWindow(
+    "key removed from window",
+    Seq( Seq(("a", 1)), Seq(("a", 1)), Seq(), Seq() ),
+    Seq( Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 1)), Seq() )
+  )
+
+  testReduceByKeyAndWindow(
+    "larger slide time",
+    largerSlideInput,
+    largerSlideReduceOutput,
+    Seconds(4),
+    Seconds(2)
+  )
+
+  testReduceByKeyAndWindow("big test", bigInput, bigReduceOutput)
+
+  // Testing reduceByKeyAndWindow (with invertible reduce function)
+
+  testReduceByKeyAndWindowWithInverse(
+    "basic reduction",
+    Seq(Seq(("a", 1), ("a", 3)) ),
+    Seq(Seq(("a", 4)) )
+  )
+
+  testReduceByKeyAndWindowWithInverse(
+    "key already in window and new value added into window",
+    Seq( Seq(("a", 1)), Seq(("a", 1)) ),
+    Seq( Seq(("a", 1)), Seq(("a", 2)) )
+  )
+
+  testReduceByKeyAndWindowWithInverse(
+    "new key added into window",
+    Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ),
+    Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) )
+  )
+
+  testReduceByKeyAndWindowWithInverse(
+    "key removed from window",
+    Seq( Seq(("a", 1)), Seq(("a", 1)), Seq(), Seq() ),
+    Seq( Seq(("a", 1)), Seq(("a", 2)), Seq(("a", 1)), Seq(("a", 0)) )
+  )
+
+  testReduceByKeyAndWindowWithInverse(
+    "larger slide time",
+    largerSlideInput,
+    largerSlideReduceOutput,
+    Seconds(4),
+    Seconds(2)
+  )
+
+  testReduceByKeyAndWindowWithInverse("big test", bigInput, bigReduceInvOutput)
+
+  testReduceByKeyAndWindowWithFilteredInverse("big test", bigInput, bigReduceOutput)
+
+  test("groupByKeyAndWindow") {
+    val input = bigInput
+    val expectedOutput = bigGroupByOutput.map(_.map(x => (x._1, x._2.toSet)))
+    val windowDuration = Seconds(2)
+    val slideDuration = Seconds(1)
+    val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+    val operation = (s: DStream[(String, Int)]) => {
+      s.groupByKeyAndWindow(windowDuration, slideDuration)
+       .map(x => (x._1, x._2.toSet))
+       .persist()
+    }
+    testOperation(input, operation, expectedOutput, numBatches, true)
+  }
+
+  test("countByWindow") {
+    val input = Seq(Seq(1), Seq(1), Seq(1, 2), Seq(0), Seq(), Seq() )
+    val expectedOutput = Seq( Seq(1), Seq(2), Seq(3), Seq(3), Seq(1), Seq(0))
+    val windowDuration = Seconds(2)
+    val slideDuration = Seconds(1)
+    val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+    val operation = (s: DStream[Int]) => {
+      s.countByWindow(windowDuration, slideDuration).map(_.toInt)
+    }
+    testOperation(input, operation, expectedOutput, numBatches, true)
+  }
+
+  test("countByValueAndWindow") {
+    val input = Seq(Seq("a"), Seq("b", "b"), Seq("a", "b"))
+    val expectedOutput = Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 2)), Seq(("a", 1), ("b", 3)))
+    val windowDuration = Seconds(2)
+    val slideDuration = Seconds(1)
+    val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+    val operation = (s: DStream[String]) => {
+      s.countByValueAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt))
+    }
+    testOperation(input, operation, expectedOutput, numBatches, true)
+  }
+
+
+  // Helper functions
+
+  def testWindow(
+    name: String,
+    input: Seq[Seq[Int]],
+    expectedOutput: Seq[Seq[Int]],
+    windowDuration: Duration = Seconds(2),
+    slideDuration: Duration = Seconds(1)
+    ) {
+    test("window - " + name) {
+      val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+      val operation = (s: DStream[Int]) => s.window(windowDuration, slideDuration)
+      testOperation(input, operation, expectedOutput, numBatches, true)
+    }
+  }
+
+  def testReduceByKeyAndWindow(
+    name: String,
+    input: Seq[Seq[(String, Int)]],
+    expectedOutput: Seq[Seq[(String, Int)]],
+    windowDuration: Duration = Seconds(2),
+    slideDuration: Duration = Seconds(1)
+    ) {
+    test("reduceByKeyAndWindow - " + name) {
+      logInfo("reduceByKeyAndWindow - " + name)
+      val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+      val operation = (s: DStream[(String, Int)]) => {
+        s.reduceByKeyAndWindow((x: Int, y: Int) => x + y, windowDuration, slideDuration)
+      }
+      testOperation(input, operation, expectedOutput, numBatches, true)
+    }
+  }
+
+  def testReduceByKeyAndWindowWithInverse(
+    name: String,
+    input: Seq[Seq[(String, Int)]],
+    expectedOutput: Seq[Seq[(String, Int)]],
+    windowDuration: Duration = Seconds(2),
+    slideDuration: Duration = Seconds(1)
+  ) {
+    test("reduceByKeyAndWindow with inverse function - " + name) {
+      logInfo("reduceByKeyAndWindow with inverse function - " + name)
+      val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+      val operation = (s: DStream[(String, Int)]) => {
+        s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration)
+         .checkpoint(Seconds(100)) // Large value to avoid effect of RDD checkpointing
+      }
+      testOperation(input, operation, expectedOutput, numBatches, true)
+    }
+  }
+
+  def testReduceByKeyAndWindowWithFilteredInverse(
+      name: String,
+      input: Seq[Seq[(String, Int)]],
+      expectedOutput: Seq[Seq[(String, Int)]],
+      windowDuration: Duration = Seconds(2),
+      slideDuration: Duration = Seconds(1)
+    ) {
+    test("reduceByKeyAndWindow with inverse and filter functions - " + name) {
+      logInfo("reduceByKeyAndWindow with inverse and filter functions - " + name)
+      val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
+      val filterFunc = (p: (String, Int)) => p._2 != 0
+      val operation = (s: DStream[(String, Int)]) => {
+        s.reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration, filterFunc = filterFunc)
+          .persist()
+          .checkpoint(Seconds(100)) // Large value to avoid effect of RDD checkpointing
+      }
+      testOperation(input, operation, expectedOutput, numBatches, true)
+    }
+  }
+}