diff --git a/.gitignore b/.gitignore
index 88d7b56181be7607e315b5f01990ed8951336dc9..155e785b01beb809a13c45c40d96f04f4dd6343b 100644
--- a/.gitignore
+++ b/.gitignore
@@ -34,3 +34,5 @@ log/
 spark-tests.log
 streaming-tests.log
 dependency-reduced-pom.xml
+.ensime
+.ensime_lucene
diff --git a/bagel/pom.xml b/bagel/pom.xml
index a8256a6e8b0ea32bc028b3404c40fa70b81363e2..667d28c1a2ba2b1fa730c01dc0bc025acde846f5 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.spark-project</groupId>
     <artifactId>parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.7.1-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
diff --git a/core/pom.xml b/core/pom.xml
index 66c62151feb757d5da56edaf9c53eb8028a4675e..9d46d94c1cce328baf27cf5ad51c5821a8be7d94 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.spark-project</groupId>
     <artifactId>parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.7.1-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index 4319cbd892a374f5d90c5a29251297a370a77902..e7408e4352abfc2e53e20f91039e186ff72c139f 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.
@@ -248,8 +249,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)] = {
@@ -259,7 +260,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)
@@ -267,7 +269,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))
@@ -295,7 +297,7 @@ 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
-   * 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))
@@ -315,7 +317,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
    * 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))
@@ -438,17 +440,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
     cogroup(other1, other2, defaultPartitioner(self, other1, other2))
   }
 
-  /**
-   * 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.
-   */
-  def defaultPartitioner(rdds: RDD[_]*): Partitioner = {
-    for (r <- rdds if r.partitioner != None) {
-      return r.partitioner.get
-    }
-    return new HashPartitioner(self.context.defaultParallelism)
-  }
-
   /**
    * Return the list of values in the RDD for key `key`. This operation is done efficiently if the
    * RDD has a known partitioner by only searching the partition that the key maps to.
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 da82dfd10f290b70c7f8a2244b2065014acee558..584efa8adf5f3877c914cf54b0fd22eefc325963 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -16,6 +16,7 @@ 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
@@ -30,6 +31,7 @@ import spark.rdd.MapPartitionsRDD
 import spark.rdd.MapPartitionsWithIndexRDD
 import spark.rdd.PipedRDD
 import spark.rdd.SampledRDD
+import spark.rdd.SubtractedRDD
 import spark.rdd.UnionRDD
 import spark.rdd.ZippedRDD
 import spark.storage.StorageLevel
@@ -299,19 +301,26 @@ abstract class RDD[T: ClassManifest](
    */
   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, numPartitions: Int): RDD[(K, Seq[T])] = {
-    val cleanF = sc.clean(f)
-    this.map(t => (cleanF(t), t)).groupByKey(numPartitions)
-  }
+  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.
@@ -393,6 +402,26 @@ abstract class RDD[T: ClassManifest](
     filter(f.isDefinedAt).map(f)
   }
 
+  /**
+   * 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] = new SubtractedRDD[T](this, other, p)
+
   /**
    * Reduces the elements of this RDD using the specified commutative and associative binary operator.
    */
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index e1e2682f8a8a7151666b47d1f74382754b08590b..4957a54c1b8af5c199a3f7ffd235c1ff492e0033 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -59,7 +59,7 @@ class SparkContext(
     val appName: String,
     val sparkHome: String = null,
     val jars: Seq[String] = Nil,
-    environment: Map[String, String] = Map())
+    val environment: Map[String, String] = Map())
   extends Logging {
 
   // Ensure logging is initialized before we spawn any threads
@@ -434,7 +434,7 @@ 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)
@@ -696,7 +696,7 @@ class SparkContext(
     checkpointDir = Some(dir)
   }
 
-  /** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */
+  /** 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 partitions for Hadoop RDDs when not given by user */
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
index da3cb2cd31395e022274a8010339546cacf949b8..ba00b6a8448f1d28d2bd4d257aca9a62db8b7539 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] {
 
@@ -57,6 +57,27 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
    */
   def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(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: 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 df3af3817dc812a24288acf19ec07744e52fd84e..c1bd13c49a9e64a0dcbbad35fa2c1e9f02c32d81 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
 
@@ -59,7 +60,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   /**
    * 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()))
 
   /**
@@ -102,7 +103,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] = {
@@ -181,6 +182,27 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
   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 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`
    * is true, Spark will group values of the same key together on the map side before the
@@ -220,30 +242,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()))
@@ -268,7 +290,7 @@ 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
-   * 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))
@@ -286,7 +308,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
    * 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))
@@ -309,7 +331,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))
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala
index 3ccd6f055ebef53804e0220c82e8ff69e054c325..301688889898e169e52b75951c159fb1b7a3159d 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDD.scala
@@ -55,6 +55,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 90b45cf875ea7fb42b8bde2ffbade7d37db1dff4..d884529d7a6f552227deb3989912efeff13cd5f2 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -12,7 +12,7 @@ import spark.storage.StorageLevel
 import com.google.common.base.Optional
 
 
-trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround[T] {
+trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
   def wrapRDD(rdd: RDD[T]): This
 
   implicit val classManifest: ClassManifest[T]
@@ -82,12 +82,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
   }
 
   /**
-   * Part of the workaround for SPARK-668; called in PairFlatMapWorkaround.java.
+   *  Return a new RDD by first applying a function to all elements of this
+   *  RDD, and then flattening the results.
    */
-  private[spark] def doFlatMap[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 PairFlatMapWorkaround
   /**
    * 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())
   }
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index f75fc27c7b2f63d024aea676bae8df9de55ded7e..5f18b1e15bd69c866e0e52b19f7b8b1e2303c13d 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -31,8 +31,8 @@ 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 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, appName: String, sparkHome: String, jarFile: String) =
     this(new SparkContext(master, appName, sparkHome, Seq(jarFile)))
diff --git a/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java b/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java
deleted file mode 100644
index 68b6fd6622742148761363045c6a06d0e8afeb74..0000000000000000000000000000000000000000
--- a/core/src/main/scala/spark/api/java/PairFlatMapWorkaround.java
+++ /dev/null
@@ -1,20 +0,0 @@
-package spark.api.java;
-
-import spark.api.java.JavaPairRDD;
-import spark.api.java.JavaRDDLike;
-import spark.api.java.function.PairFlatMapFunction;
-
-import java.io.Serializable;
-
-/**
- * Workaround for SPARK-668.
- */
-class PairFlatMapWorkaround<T> implements Serializable {
-    /**
-     *  Return a new RDD by first applying a function to all elements of this
-     *  RDD, and then flattening the results.
-     */
-    public <K, V> JavaPairRDD<K, V> flatMap(PairFlatMapFunction<T, K, V> f) {
-        return ((JavaRDDLike <T, ?>) this).doFlatMap(f);
-    }
-}
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala
index 8c734773847b5d5ec12506b34dcd9e577b9cf930..9b4d54ab4e0461364d643bb857b267ed4e05bed6 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/spark/api/python/PythonRDD.scala
@@ -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,7 +60,7 @@ 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)
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index 1cd68a2aa61ddcc874fd0245dd5b0eb52f4f3479..4af22cf9b6fcd65fd6007dd99d27f7ed37f46669 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -33,6 +33,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
   val waitingApps = new ArrayBuffer[ApplicationInfo]
   val completedApps = new ArrayBuffer[ApplicationInfo]
 
+  var firstApp: Option[ApplicationInfo] = None
+
   val masterPublicAddress = {
     val envVar = System.getenv("SPARK_PUBLIC_DNS")
     if (envVar != null) envVar else ip
@@ -41,7 +43,7 @@ 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 app
   // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
-  val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "false").toBoolean
+  val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
 
   override def preStart() {
     logInfo("Starting Spark master at spark://" + ip + ":" + port)
@@ -167,7 +169,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
       // 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(app, _)).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(app.coresLeft, usableWorkers.map(_.coresFree).sum)
@@ -190,7 +192,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
       }
     } else {
       // Pack each app into as few nodes as possible until we've assigned all its cores
-      for (worker <- workers if worker.coresFree > 0) {
+      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)
@@ -245,6 +247,13 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
     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
   }
 
@@ -254,7 +263,7 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
       apps -= app
       idToApp -= app.id
       actorToApp -= app.driver
-      addressToWorker -= app.driver.path.address
+      addressToApp -= app.driver.path.address
       completedApps += app   // Remember it in our history
       waitingApps -= app
       for (exec <- app.executors.values) {
diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala
index cd5b7d57f32f58c3c1a5e20e1dcc011d0650a7e1..d1451bc2124c581eff01dfae5277612ea5c995c7 100644
--- a/core/src/main/scala/spark/network/Connection.scala
+++ b/core/src/main/scala/spark/network/Connection.scala
@@ -198,7 +198,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
     outbox.synchronized {
       outbox.addMessage(message)
       if (channel.isConnected) {
-        changeConnectionKeyInterest(SelectionKey.OP_WRITE)
+        changeConnectionKeyInterest(SelectionKey.OP_WRITE | SelectionKey.OP_READ)
       }
     }
   }
@@ -219,7 +219,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
   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 => {
@@ -239,8 +239,7 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
                 currentBuffers ++= chunk.buffers 
               }
               case None => {
-                changeConnectionKeyInterest(0)
-                /*key.interestOps(0)*/
+                changeConnectionKeyInterest(SelectionKey.OP_READ)
                 return
               }
             }
@@ -267,6 +266,23 @@ extends Connection(SocketChannel.open, selector_, remoteId_) {
       }
     }
   }
+
+  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/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
index 36bfb0355e0e17e487aa3d4c7aea5bc4cc54e844..9e37bdf659201c2ec7be7dd84de83881d34d8704 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -22,10 +22,10 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
 
   override def getPartitions: Array[Partition] = {
     val dirContents = fs.listStatus(new Path(checkpointPath))
-    val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted
-    val numPartitions = splitFiles.size
-    if (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) ||
-        !splitFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1))) {
+    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))
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index 8139a2a40c66fc9a2f0f24adc09df30ed2df8c5f..78097502bca48d207a65563718079c638490d987 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -15,7 +15,7 @@ import org.apache.hadoop.mapred.RecordReader
 import org.apache.hadoop.mapred.Reporter
 import org.apache.hadoop.util.ReflectionUtils
 
-import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext}
+import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
 
 
 /**
@@ -42,7 +42,7 @@ class HadoopRDD[K, V](
     keyClass: Class[K],
     valueClass: Class[V],
     minSplits: Int)
-  extends RDD[(K, V)](sc, Nil) {
+  extends RDD[(K, V)](sc, Nil) with Logging {
 
   // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
   private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@@ -71,7 +71,7 @@ class HadoopRDD[K, V](
     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{ () => close() }
 
     val key: K = reader.createKey()
     val value: V = reader.createValue()
@@ -88,9 +88,6 @@ class HadoopRDD[K, V](
         }
         gotNext = true
       }
-      if (finished) {
-        reader.close()
-      }
       !finished
     }
 
@@ -104,6 +101,14 @@ class HadoopRDD[K, V](
       gotNext = false
       (key, value)
     }
+
+    private def close() {
+      try {
+        reader.close()
+      } catch {
+        case e: Exception => logWarning("Exception in RecordReader.close()", e)
+      }
+    }
   }
 
   override def getPreferredLocations(split: Partition): Seq[String] = {
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index ebd4c3f0e2d863ddfdf629b5666add02f182ee55..df2361025c75327837c4e13184e762261e4b7509 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -7,7 +7,7 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.Writable
 import org.apache.hadoop.mapreduce._
 
-import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext}
+import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext}
 
 
 private[spark]
@@ -26,7 +26,8 @@ class NewHadoopRDD[K, V](
     valueClass: Class[V],
     @transient conf: Configuration)
   extends RDD[(K, V)](sc, Nil)
-  with HadoopMapReduceUtil {
+  with HadoopMapReduceUtil
+  with Logging {
 
   // A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
   private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@@ -61,7 +62,7 @@ class NewHadoopRDD[K, V](
     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
@@ -81,6 +82,14 @@ 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 getPreferredLocations(split: Partition): Seq[String] = {
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..daf9cc993cf42e9e963e986d73cdad0d6d708059
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/SubtractedRDD.scala
@@ -0,0 +1,108 @@
+package spark.rdd
+
+import java.util.{HashSet => JHashSet}
+import scala.collection.JavaConversions._
+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[T: ClassManifest](
+    @transient var rdd1: RDD[T],
+    @transient var rdd2: RDD[T],
+    part: Partitioner) extends RDD[T](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)
+        val mapSideCombinedRDD = rdd.mapPartitions(i => {
+          val set = new JHashSet[T]()
+          while (i.hasNext) {
+            set.add(i.next)
+          }
+          set.iterator
+        }, true)
+        // ShuffleDependency requires a tuple (k, v), which it will partition by k.
+        // We need this to partition to map to the same place as the k for
+        // OneToOneDependency, which means:
+        // - for already-tupled RDD[(A, B)], into getPartition(a)
+        // - for non-tupled RDD[C], into getPartition(c)
+        val part2 = new Partitioner() {
+          def numPartitions = part.numPartitions
+          def getPartition(key: Any) = key match {
+            case (k, v) => part.getPartition(k)
+            case k => part.getPartition(k)
+          }
+        }
+        new ShuffleDependency(mapSideCombinedRDD.map((_, null)), part2)
+      }
+    }
+  }
+
+  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))
+        }
+      }.toList)
+    }
+    array
+  }
+
+  override val partitioner = Some(part)
+
+  override def compute(p: Partition, context: TaskContext): Iterator[T] = {
+    val partition = p.asInstanceOf[CoGroupPartition]
+    val set = new JHashSet[T]
+    def integrate(dep: CoGroupSplitDep, op: T => Unit) = dep match {
+      case NarrowCoGroupSplitDep(rdd, _, itsSplit) =>
+        for (k <- rdd.iterator(itsSplit, context))
+          op(k.asInstanceOf[T])
+      case ShuffleCoGroupSplitDep(shuffleId) =>
+        for ((k, _) <- SparkEnv.get.shuffleFetcher.fetch(shuffleId, partition.index))
+          op(k.asInstanceOf[T])
+    }
+    // the first dep is rdd1; add all keys to the set
+    integrate(partition.deps(0), set.add)
+    // the second dep is rdd2; remove all of its keys from the set
+    integrate(partition.deps(1), set.remove)
+    set.iterator
+  }
+
+  override def clearDependencies() {
+    super.clearDependencies()
+    rdd1 = null
+    rdd2 = null
+  }
+
+}
\ No newline at end of file
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
index 1e4fbdb8742fdac8d33edb6b1e9ec1b3aaaea4d4..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,6 +23,8 @@ 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]
@@ -30,6 +33,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
   val taskIdToExecutorId = new HashMap[Long, String]
   val taskSetTaskIds = new HashMap[String, HashSet[Long]]
 
+  var hasReceivedTask = false
+  var hasLaunchedTask = false
+  val starvationTimer = new Timer(true)
+
   // Incrementing Mesos task IDs
   val nextTaskId = new AtomicLong(0)
 
@@ -94,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()
   }
@@ -119,6 +140,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
       // Mark each slave as alive and remember its hostname
       for (o <- offers) {
         executorIdToHost(o.executorId) = o.hostname
+        if (!executorsByHost.contains(o.hostname)) {
+          executorsByHost(o.hostname) = new HashSet()
+        }
       }
       // Build a list of tasks to assign to each slave
       val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores))
@@ -138,9 +162,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
                 taskSetTaskIds(manager.taskSet.id) += tid
                 taskIdToExecutorId(tid) = execId
                 activeExecutorIds += execId
-                if (!executorsByHost.contains(host)) {
-                  executorsByHost(host) = new HashSet()
-                }
                 executorsByHost(host) += execId
                 availableCpus(i) -= 1
                 launchedTask = true
@@ -150,6 +171,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
           }
         } while (launchedTask)
       }
+      if (tasks.size > 0) {
+        hasLaunchedTask = true
+      }
       return tasks
     }
   }
@@ -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
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index d6064325724df13d34be05e90772d2bb58994947..7a428e3361976b3a8d78b8a8c1c1a7f771399c1a 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -153,7 +153,8 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
     driverActor ! ReviveOffers
   }
 
-  override 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) {
diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
index 7389bee1509d45cceb61e9afce0760e08de01343..d93cfc48d03fc563dd3bf91ed8c8a3c45ea24b14 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
@@ -26,7 +26,7 @@ private[spark] class BlockManagerMaster(
   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 DRIVER_AKKA_ACTOR_NAME = "BlockMasterManager"
+  val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
 
   val timeout = 10.seconds
   var driverActor: ActorRef = {
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala
index ae88ff0bb1694c89a580bdd89bf127991ecb7f1e..949588476c20150b1dd5c73f4303dbf85d2ad518 100644
--- a/core/src/main/scala/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/spark/storage/MemoryStore.scala
@@ -32,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
@@ -58,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/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index 30aec5a663040505398a221b892104d77cd0baea..3e805b78314c78267e86fa2c6f9a0e359637f624 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/spark/util/AkkaUtils.scala
@@ -31,20 +31,22 @@ private[spark] object AkkaUtils {
     val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
     val akkaTimeout = System.getProperty("spark.akka.timeout", "20").toInt
     val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
+    val lifecycleEvents = System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean
     val 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.log-remote-lifecycle-events = on
       akka.remote.netty.hostname = "%s"
       akka.remote.netty.port = %d
       akka.remote.netty.connection-timeout = %ds
       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(name, akkaConf, getClass.getClassLoader)
 
diff --git a/core/src/main/scala/spark/util/MetadataCleaner.scala b/core/src/main/scala/spark/util/MetadataCleaner.scala
index a342d378ffab263f8dc23b58ec7e7f1f204bd045..dafa90671214b4d803f2775534ebee9f5325cc70 100644
--- a/core/src/main/scala/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/spark/util/MetadataCleaner.scala
@@ -38,7 +38,7 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
 
 
 object MetadataCleaner {
-  def getDelaySeconds = System.getProperty("spark.cleaner.delay", "-1").toInt
-  def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.delay", delay.toString) }
+  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/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/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala
index 3e5ffa81d6f8f288859413a40c6ed194c111df34..ca385972fb2ebe3add71881f18c3edc9761077d8 100644
--- a/core/src/test/scala/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/spark/CheckpointSuite.scala
@@ -162,6 +162,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
       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.
diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala
index 0e2585daa434cdad2c8deb14bd088072a69dfe31..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,7 +14,7 @@ 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 with LocalSparkContext {
 
@@ -140,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") {
@@ -217,6 +231,27 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
       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 {
diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala
index af1107cd197b4f7c6221bd66bf9beee9ec7270f8..60db759c25f3b424280e869df5319b21c2c3e7e6 100644
--- a/core/src/test/scala/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/spark/PartitioningSuite.scala
@@ -84,10 +84,10 @@ class PartitioningSuite extends FunSuite with LocalSparkContext {
     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/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 92c3f6741668b510af22020d8d34866cb5f11a7c..8411291b2caa31e86f58bbf17f39fbf68020a669 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -234,6 +234,51 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
     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)
+    println(sc.runJob(a, (i: Iterator[(Int, String)]) => i.toList).toList)
+    // more partitions/no partitioner so a shuffle dependency 
+    val b = sc.parallelize(Array((2, "b"), (3, "cc"), (4, "d")), 4)
+    val c = a.subtract(b)
+    assert(c.collect().toSet === Set((1, "a"), (3, "c")))
+    assert(c.partitioner.get === p)
+  }
 }
 
 object ShuffleSuite {
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 09617e4a1efb6b3486970b7fc7715f1fb0993a16..f99d5bb376027823a57749e6e5e07b4ac04a69f7 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -3,8 +3,8 @@ markdown: kramdown
 
 # These allow the documentation to be updated with nerw releases
 # of Spark, Scala, and Mesos.
-SPARK_VERSION: 0.7.0-SNAPSHOT
-SPARK_VERSION_SHORT: 0.7.0
+SPARK_VERSION: 0.7.1-SNAPSHOT
+SPARK_VERSION_SHORT: 0.7.1
 SCALA_VERSION: 2.9.2
 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 94baa634aaf4e640c3daaf2d82e6b2a9b23c2e51..280ead03232adaa7da876b8016d0e779ef5c8a33 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]>
@@ -53,11 +69,11 @@
                         </li>
                         
                         <li class="dropdown">
-                            <a href="#" class="dropdown-toggle" data-toggle="dropdown">API (Scaladoc)<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">Spark Scala/Java (Scaladoc)</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 Scala/Java (Scaladoc) </a></li>
+                                <li><a href="api/streaming/index.html">Spark Streaming Java/Scala (Scaladoc) </a></li>
                             </ul>
                         </li>
 
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index e400dec619f772a6466f0ac903217dd34e37c1fa..d77e53963c2f7594e5857103bf062c810b861023 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -1,7 +1,8 @@
 require 'fileutils'
 include FileUtils
 
-if ENV['SKIP_SCALADOC'] != '1'
+if ENV['SKIP_API'] != '1'
+  # Build Scaladoc for Java/Scala
   projects = ["core", "examples", "repl", "bagel", "streaming"]
 
   puts "Moving to project root and building scaladoc."
@@ -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/configuration.md b/docs/configuration.md
index f1ca77aa7826a9da39153d6ffe4ac74198b65a0f..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>
@@ -244,10 +251,10 @@ Apart from these, the following properties are also available, and may be useful
   </td>
 </tr>
 <tr>
-  <td>spark.cleaner.delay</td>
+  <td>spark.cleaner.ttl</td>
   <td>(disable)</td>
   <td>
-    Duration (minutes) of how long Spark will remember any metadata (stages generated, tasks generated, etc.).
+    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.
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 931b7a66bdcc17bc64a0c11d58ca6407bd5dc5b7..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
@@ -106,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.
 
@@ -152,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 c6ef507cb0997189155ed8eb20799e615e765d40..45facd8e63f32494edb5759f2b2f8eb46421c17c 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
 
@@ -58,9 +52,9 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
 
 * [Quick Start](quick-start.html): a quick introduction to the Spark API; start here!
 * [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API
-* [Streaming Programming Guide](streaming-programming-guide.html): an API preview of Spark Streaming
 * [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:**
 
@@ -92,7 +86,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 37a906ea1c780687474849473db056ad9ff71ef8..ae8257b53938e0672efc1cb35d4f77893a7a1ec2 100644
--- a/docs/java-programming-guide.md
+++ b/docs/java-programming-guide.md
@@ -189,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 4e84d23edf56b0a6b6fe3291744c269be0085b28..3a7a8db4a6ee43fdfa7af612c39a09b953b6560a 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -87,7 +87,7 @@ 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 %}
 
@@ -109,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..36d024f13ac1807654611c0b01800b4d9039bf61 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -136,7 +136,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 %}
 
@@ -207,8 +207,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>
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index b98718a5532e0d49e19437be39bc7482f7d6d328..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
 
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
index b6da7af654ee6de2e5ae5733faf1c69ad8a8092b..b30699cf3df8615b0e9360a751849aed76da3342 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -7,9 +7,9 @@ title: Spark Streaming Programming Guide
 {: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 collection of elements) representing a continuous stream of data. DStreams can created from live incoming data (such as data from a socket, Kafka, etc.) or it can be generated by transformation of 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 the data into batches. Each batch of data is treated as a RDD, that is a immutable and parallel collection of data. These input data RDDs are automatically persisted in memory (serialized by default) and replicated to two nodes for fault-tolerance. This sequence of RDDs is collectively referred to as an InputDStream.
-(ii) Data received by InputDStreams are processed 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.  
+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. 
 
@@ -17,16 +17,12 @@ This guide shows some how to start programming with DStreams.
 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, jobName, batchDuration)
+new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
 {% endhighlight %}
 
-The `master` parameter is either the [Mesos master URL](running-on-mesos.html) (for running on a cluster)or the special "local" string (for local mode) that is used to create a Spark Context. For more information about this please refer to the [Spark programming guide](scala-programming-guide.html). The `jobName` is the name of the streaming job, which is the same as the jobName used in SparkContext. It is used to identify this job in the Mesos 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. Starting with something conservative like 5 seconds maybe a good start. See [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion.
-
-This constructor creates a SparkContext object using the given `master` and `jobName` parameters. However, if you already have a SparkContext or you need to create a custom SparkContext by specifying list of JARs, then a StreamingContext can be created from the existing SparkContext, by using
-{% highlight scala %}
-new StreamingContext(sparkContext, batchDuration)
-{% 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
@@ -34,34 +30,34 @@ The StreamingContext is used to creating InputDStreams from input sources:
 
 {% highlight scala %}
 // Assuming ssc is the StreamingContext
-ssc.networkStream(hostname, port)    // Creates a stream that uses a TCP socket to read data from hostname:port
-ssc.textFileStream(directory)   // Creates a stream by monitoring and processing new files in a HDFS directory
+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 %}
 
-A complete list of input sources is available in the [StreamingContext API documentation](api/streaming/index.html#spark.streaming.StreamingContext). Data received from these sources can be processed using DStream operations, which are explained next.
+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
-Once an input DStream has been created, you can transform it using _DStream operators_. Most of these operators return new DStreams which you can further transform. Eventually, you'll need to call an _output operator_, which forces evaluation of the DStream by writing data out to an external source.
+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:25%">Transformation</th><th>Meaning</th></tr>
+<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 through a function <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 stream formed by selecting those elements of the source on which <i>func</i> returns true. </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 Seq rather than a single item). </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>
@@ -70,73 +66,92 @@ DStreams support many of the transformations available on normal Spark RDD's:
 </tr>
 <tr>
   <td> <b>union</b>(<i>otherStream</i>) </td>
-  <td> Return a new stream that contains the union of the elements in the source stream and the argument. </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 stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs. <br />
-<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
+  <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together all the values of each key in the RDDs of the source DStream. <br />
+  <b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
 </td>
 </tr>
 <tr>
   <td> <b>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td>
-  <td> When called on a stream of (K, V) pairs, returns a stream 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>
+  <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 streams of type (K, V) and (K, W), returns a stream of (K, (V, W)) pairs with all pairs of elements for each key. </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 type (K, V) and (K, W), returns a DStream of (K, Seq[V], Seq[W]) tuples.</td>
-</tr>
-<tr>
-  <td> <b>reduce</b>(<i>func</i>) </td>
-  <td> Returns a new DStream of single-element RDDs by aggregating the elements of the stream using a function func (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. </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 report statistics 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.
+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:25%">Transformation</th><th>Meaning</th></tr>
+<tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
 <tr>
-  <td> <b>window</b>(<i>windowDuration</i>, </i>slideTime</i>) </td>
-  <td> Return a new stream which is computed based on windowed batches of the source stream. <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> <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>slideTime</i>) </td>
+  <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
   <td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
   </td>
 </tr>
 <tr>
-  <td> <b>reduceByWindow</b>(<i>func</i>, <i>windowDuration</i>, </i>slideDuration</i>) </td>
+  <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>(windowDuration, slideDuration, [<i>numTasks</i>])
+  <td> <b>groupByKeyAndWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>, [<i>numTasks</i>])
   </td>
-  <td> When called on a stream of (K, V) pairs, returns a stream of (K, Seq[V]) pairs over a sliding window. <br />
-<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
-</td>
+  <td> When called on a DStream of (K, V) pairs, returns a new DStream of (K, Seq[V]) pairs by grouping together values of each key over batches in a sliding window. <br />
+<b>Note:</b> By default, this uses Spark's default number of parallel tasks (2 for local machine, 8 for a cluser) to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.</td>
 </tr>
 <tr>
-  <td> <b>reduceByKeyAndWindow</b>(<i>func</i>, [<i>numTasks</i>]) </td>
-  <td> When called on a stream of (K, V) pairs, returns a stream of (K, V) pairs where the values for each key are aggregated using the given reduce function over batches within a sliding window. Like in <code>groupByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument. 
+  <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>countByKeyAndWindow</b>([<i>numTasks</i>]) </td>
-  <td> When called on a stream of (K, V) pairs, returns a stream of (K, Int) pairs where the values for each key are the count within a sliding window. Like in <code>countByKeyAndWindow</code>, the number of reduce tasks is configurable through an optional second argument. 
+  <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> 
+</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>
@@ -147,7 +162,7 @@ A complete list of DStream operations is available in the API documentation of [
 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:25%">Operator</th><th>Meaning</th></tr>
+<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>
@@ -176,11 +191,6 @@ When an output operator is called, it triggers the computation of a stream. Curr
 
 </table>
 
-## 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 DStream 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()`.
-
-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).
-
 # 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 %}
@@ -192,8 +202,8 @@ Conversely, the computation can be stopped by using
 ssc.stop()
 {% endhighlight %}
 
-# Example - NetworkWordCount.scala
-A good example to start off is the spark.streaming.examples.NetworkWordCount. 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/WordCountNetwork.scala.
+# 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}
@@ -202,7 +212,7 @@ 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.networkTextStream(args(1), args(2).toInt)
+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(" "))
@@ -213,6 +223,8 @@ wordCounts.print()
 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 %}
@@ -260,6 +272,33 @@ Time: 1357008430000 ms
 </td>
 </table>
 
+You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run 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
@@ -273,17 +312,21 @@ Getting the best performance of a Spark Streaming application on a cluster requi
 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 underutilized 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.
+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.
+
+* **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.
+
+* **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
@@ -292,22 +335,183 @@ For a Spark Streaming application running on a cluster to be stable, the process
 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.delay` to the number of minutes you want any metadata to persist. For example, setting `spark.cleaner.delay` to 10 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.
+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.
 
-* <b>Default persistence level of DStreams</b>: 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.
+* **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
 
-* <b>Concurrent garbage collector</b>: 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.
+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.
 
-# Master Fault-tolerance (Alpha)
-TODO
+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 %}
 
-* Checkpointing of DStream graph
 
-* Recovery from master faults
 
-* Current state and future directions
\ No newline at end of file
+# 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 738c530458b59a24047b18982748fd6a4ac26ddd..843380b9a28829898877dd092018e1b31ddabca5 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -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.
 
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/spark_ec2.py b/ec2/spark_ec2.py
index 66b1faf2cd8314d54ca4bd0ba9f2c65f0663ba55..571d27fde66ed21d00aacdda1f587af381e5fefb 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
@@ -83,16 +83,16 @@ def parse_args():
       help="If specified, launch slaves as spot instances with the given " +
             "maximum price (in dollars)")
   parser.add_option("--cluster-type", type="choice", metavar="TYPE",
-      choices=["mesos", "standalone"], default="mesos",
+      choices=["mesos", "standalone"], default="standalone",
       help="'mesos' for a Mesos cluster, 'standalone' for a standalone " +
-           "Spark cluster (default: mesos)")
+           "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("--new-scripts", action="store_true", default=False,
-      help="Use new spark-ec2 scripts, for Spark >= 0.7 AMIs")
+  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)")
   parser.add_option("--delete-groups", action="store_true", default=False,
@@ -383,7 +383,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k
   if opts.ganglia:
     modules.append('ganglia')
 
-  if opts.new_scripts:
+  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")
@@ -393,7 +393,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, zoo_nodes, opts, deploy_ssh_k
           zoo_nodes, modules)
 
   print "Running setup on master..."
-  if not opts.new_scripts:
+  if opts.old_scripts:
     if opts.cluster_type == "mesos":
       setup_mesos_cluster(master, opts)
     elif opts.cluster_type == "standalone":
diff --git a/examples/pom.xml b/examples/pom.xml
index f43af670c613fb6fa3dcaa09717b3e16e83af6da..2adeec8786fb378df9b912df0c3b57feee6df141 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.spark-project</groupId>
     <artifactId>parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.7.1-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -20,11 +20,10 @@
       <artifactId>jetty-server</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.twitter4j</groupId>
-      <artifactId>twitter4j-stream</artifactId>
-      <version>3.0.3</version>
+      <groupId>com.twitter</groupId>
+      <artifactId>algebird-core_2.9.2</artifactId>
+      <version>0.1.8</version>
     </dependency>
-
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.version}</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/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/scala/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
similarity index 93%
rename from examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java
rename to examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
index cddce16e396cdb1ff7f086a05cfe0ae028d39c83..e24c6ddaa79296ff98b5508597fa0d146981befa 100644
--- a/examples/src/main/scala/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/spark/streaming/examples/JavaFlumeEventCount.java
@@ -32,7 +32,8 @@ public class JavaFlumeEventCount {
 
     Duration batchInterval = new Duration(2000);
 
-    JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval);
+    JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
+            System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
 
     JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
 
diff --git a/examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
similarity index 87%
rename from examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java
rename to examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
index 4299febfd6ca3b57569ea1b22e772015975ec8de..3e57580fd4e6c4613dbc30d274824f5e052ad9c7 100644
--- a/examples/src/main/scala/spark/streaming/examples/JavaNetworkWordCount.java
+++ b/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java
@@ -23,19 +23,19 @@ import spark.streaming.api.java.JavaStreamingContext;
  */
 public class JavaNetworkWordCount {
   public static void main(String[] args) {
-    if (args.length < 2) {
+    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));
+    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.networkTextStream(args[1], Integer.parseInt(args[2]));
+    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) {
diff --git a/examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
similarity index 94%
rename from examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java
rename to examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
index 43c3cd4dfac4ba1d6dc83a44d35da0e54b579359..15b82c8da15946d486147e3d541df5bd1d6a9595 100644
--- a/examples/src/main/scala/spark/streaming/examples/JavaQueueStream.java
+++ b/examples/src/main/java/spark/streaming/examples/JavaQueueStream.java
@@ -22,7 +22,8 @@ public class JavaQueueStream {
     }
 
     // Create the context
-    JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000));
+    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
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..b07e799cef5afcfb855d64da4858326e17c4d8e7 100644
--- a/examples/src/main/scala/spark/examples/LocalKMeans.scala
+++ b/examples/src/main/scala/spark/examples/LocalKMeans.scala
@@ -6,6 +6,9 @@ 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
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
index 5330b8da9444f46d61cd2af3403f53c26da72e4a..6497596d35f34a81b07f4cbdf972d0eabcabf032 100644
--- a/examples/src/main/scala/spark/examples/LogQuery.scala
+++ b/examples/src/main/scala/spark/examples/LogQuery.scala
@@ -26,7 +26,9 @@ object LogQuery {
       System.err.println("Usage: LogQuery <master> [logFile]")
       System.exit(1)
     }
-    val sc = new SparkContext(args(0), "Log Query")
+
+    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))
diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
index 83ae014e9458063c41a15e861c1ee638b43f8ca4..92cd81c48742fb7fe1e7c598512d5bd91e0dc5ee 100644
--- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
@@ -9,21 +9,25 @@ object MultiBroadcastTest {
       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)
     }
     
diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
index 50b3a263b4feb993dbf5b5783758b0d75aff10ec..0d17bda004b6e2f1fc4d5b43f703dd11f97c79e8 100644
--- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
@@ -18,7 +18,8 @@ object SimpleSkewedGroupByTest {
     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
diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
index d2117a263e6d8269d13619b134ed8962fd4014b7..83be3fc27b5b0a5c3ad2549f0b443136417d55a6 100644
--- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
@@ -16,13 +16,14 @@ object SkewedGroupByTest {
     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 5e01885dbb464bcdf759ffc214811a009648669a..8fb3b0fb2ad78eadbb906c8e5c11cc4131b8b929 100644
--- a/examples/src/main/scala/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/spark/examples/SparkALS.scala
@@ -1,14 +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._
-import scala.Option
 
+/**
+ * Alternating least squares matrix factorization.
+ */
 object SparkALS {
   // Parameters set through command line arguments
   var M = 0 // Number of movies
@@ -70,30 +70,32 @@ object SparkALS {
   }
 
   def main(args: Array[String]) {
+    if (args.length == 0) {
+      System.err.println("Usage: SparkALS <master> [<M> <U> <F> <iters> <slices>]")
+      System.exit(1)
+    }
+
     var host = ""
     var slices = 0
 
-    (0 to 5).map(i => {
-      i match {
-        case a if a < args.length => Some(args(a))
-        case _ => None
-      }
-    }).toArray match {
-      case Array(host_, m, u, f, iters, slices_) => {
-        host = host_ getOrElse "local"
-        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>]")
+    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")
+
+    val sc = new SparkContext(host, "SparkALS",
+      System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
     
     val R = generateR()
 
@@ -102,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 => update(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 => update(i, usc.value(i), msc.value, algebra.transpose(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..7c21ea12fb72430089d0a4166c8b74fac7677277 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
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..5a31d74444f1c637b835c907c2337d09724f59e3 100644
--- a/examples/src/main/scala/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/spark/examples/SparkPi.scala
@@ -10,7 +10,8 @@ object SparkPi {
       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
index 461929fba22ef06d36dfbfb9caa73f35edbcccaa..39c76fd98a11e3ff4b5da2f98b4b8839a1a33f9b 100644
--- a/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/FlumeEventCount.scala
@@ -30,7 +30,8 @@ object FlumeEventCount {
 
     val batchInterval = Milliseconds(2000)
     // Create the context and set the batch size
-    val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval)
+    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)
diff --git a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
index 8530f5c17549321dc70ce4aecb174fde24403e55..9389f8a38dcca41a5a6d55efe9094bb68d8b98ee 100644
--- a/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/HdfsWordCount.scala
@@ -22,7 +22,8 @@ object HdfsWordCount {
     }
 
     // Create the context
-    val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2))
+    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
diff --git a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
index fe55db6e2c6314739cf3c3e2fd4f34511aff7b6f..c3a9e491ba85406dd99262dfe1a82f6a1c42198b 100644
--- a/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/KafkaWordCount.scala
@@ -10,22 +10,34 @@ 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 < 6) {
-      System.err.println("Usage: KafkaWordCount <master> <hostname> <port> <group> <topics> <numThreads>")
+    if (args.length < 5) {
+      System.err.println("Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>")
       System.exit(1)
     }
 
-    val Array(master, hostname, port, group, topics, numThreads) = args
+    val Array(master, zkQuorum, group, topics, numThreads) = args
 
-    val sc = new SparkContext(master, "KafkaWordCount")
-    val ssc =  new StreamingContext(sc, Seconds(2))
+    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](hostname, port.toInt, group, topicpMap)
+    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()
@@ -38,16 +50,16 @@ object KafkaWordCount {
 object KafkaWordCountProducer {
 
   def main(args: Array[String]) {
-    if (args.length < 3) {
-      System.err.println("Usage: KafkaWordCountProducer <hostname> <port> <topic> <messagesPerSec> <wordsPerMessage>")
+    if (args.length < 2) {
+      System.err.println("Usage: KafkaWordCountProducer <zkQuorum> <topic> <messagesPerSec> <wordsPerMessage>")
       System.exit(1)
     }
 
-    val Array(hostname, port, topic, messagesPerSec, wordsPerMessage) = args
+    val Array(zkQuorum, topic, messagesPerSec, wordsPerMessage) = args
 
     // Zookeper connection properties
     val props = new Properties()
-    props.put("zk.connect", hostname + ":" + port)
+    props.put("zk.connect", zkQuorum)
     props.put("serializer.class", "kafka.serializer.StringEncoder")
     
     val config = new ProducerConfig(props)
diff --git a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
index 32f7d57bea485a6d87be8537635dc2793d1666e6..704540c2bf51b35cad2647a31cc134bd82b0226c 100644
--- a/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala
@@ -16,18 +16,19 @@ import spark.streaming.StreamingContext._
  */
 object NetworkWordCount {
   def main(args: Array[String]) {
-    if (args.length < 2) {
+    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))
+    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.networkTextStream(args(1), args(2).toInt)
+    val lines = ssc.socketTextStream(args(1), args(2).toInt)
     val words = lines.flatMap(_.split(" "))
     val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
     wordCounts.print()
diff --git a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
index 2a265d021d2a04828a7b9d73879fe43eba1c94cf..f450e2104018bbcc6a90b064b732b204babd2e40 100644
--- a/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
+++ b/examples/src/main/scala/spark/streaming/examples/QueueStream.scala
@@ -15,7 +15,8 @@ object QueueStream {
     }
     
     // Create the context
-    val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1))
+    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
@@ -30,10 +31,10 @@ object QueueStream {
     
     // Create and push some RDDs into
     for (i <- 1 to 30) {
-      rddQueue += ssc.sc.makeRDD(1 to 1000, 10)
+      rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10)
       Thread.sleep(1000)
     }
     ssc.stop()
     System.exit(0)
   }
-}
\ No newline at end of file
+}
diff --git a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
index 2eec777c54e8d21da0a686c3768202ba1c0a1b93..175281e0956d4c4b9d43f1ae811dae125b71c2f2 100644
--- a/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/spark/streaming/examples/RawNetworkGrep.scala
@@ -31,13 +31,14 @@ object RawNetworkGrep {
     val Array(master, IntParam(numStreams), host, IntParam(port), IntParam(batchMillis)) = args
 
     // Create the context
-    val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis))
+    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.sc)
+    RawTextHelper.warmUp(ssc.sparkContext)
 
     val rawStreams = (1 to numStreams).map(_ =>
-      ssc.rawNetworkStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
+      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))
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..483aae452b05ef5a0ef32903cdf53bae41e1a7cb
--- /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(DELTA, EPS, 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/twitter/TwitterBasic.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
similarity index 55%
rename from examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala
rename to examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
index 377bc0c98ec1f1185a97e8e559e7151e5aef7e85..9d4494c6f2759f07d13abe78e93bf7cae5194f0e 100644
--- a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterBasic.scala
+++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala
@@ -1,19 +1,19 @@
-package spark.streaming.examples.twitter
+package spark.streaming.examples
 
-import spark.streaming.StreamingContext._
 import spark.streaming.{Seconds, StreamingContext}
+import StreamingContext._
 import spark.SparkContext._
-import spark.storage.StorageLevel
 
 /**
  * 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 TwitterBasic {
+object TwitterPopularTags {
   def main(args: Array[String]) {
     if (args.length < 3) {
-      System.err.println("Usage: TwitterBasic <master> <twitter_username> <twitter_password>" +
+      System.err.println("Usage: TwitterPopularTags <master> <twitter_username> <twitter_password>" +
         " [filter1] [filter2] ... [filter n]")
       System.exit(1)
     }
@@ -21,10 +21,9 @@ object TwitterBasic {
     val Array(master, username, password) = args.slice(0, 3)
     val filters = args.slice(3, args.length)
 
-    val ssc = new StreamingContext(master, "TwitterBasic", Seconds(2))
-    val stream = new TwitterInputDStream(ssc, username, password, filters,
-      StorageLevel.MEMORY_ONLY_SER)
-    ssc.registerInputStream(stream)
+    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("#")))
 
@@ -39,22 +38,17 @@ object TwitterBasic {
 
     // Print popular hashtags
     topCounts60.foreach(rdd => {
-      if (rdd.count() != 0) {
-        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))}
-      }
+      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 => {
-      if (rdd.count() != 0) {
-        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))}
-      }
+      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/PageViewStream.scala b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
index a191321d9105801d70c76c0a557323a897b1a1ae..e226a4a73a16ba865229c355ce0826311e549ed3 100644
--- a/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -24,20 +24,20 @@ object PageViewStream {
     val port = args(2).toInt
 
     // Create the context
-    val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1))
+    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.networkTextStream(host, port)
-                        .flatMap(_.split("\n"))
-                        .map(PageView.fromString(_))
+    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, 1))).countByKey()
+    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, 1)))
-                                .window(Seconds(10), Seconds(2))
-                                .countByKey()
+    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
@@ -61,7 +61,7 @@ object PageViewStream {
                                    .map("Unique active users: " + _)
 
     // An external dataset we want to join to this stream
-    val userList = ssc.sc.parallelize(
+    val userList = ssc.sparkContext.parallelize(
        Map(1 -> "Patrick Wendell", 2->"Reynold Xin", 3->"Matei Zaharia").toSeq)
 
     metric match {
diff --git a/pom.xml b/pom.xml
index 7e06cae052b58d6ebc1e5240f4810601287d9ca2..8f4e499ee108e42d1618a321547dc0129eba6c78 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.spark-project</groupId>
   <artifactId>parent</artifactId>
-  <version>0.7.0-SNAPSHOT</version>
+  <version>0.7.1-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Spark Project Parent POM</name>
   <url>http://spark-project.org/</url>
@@ -84,9 +84,9 @@
       </snapshots>
     </repository>
     <repository>
-      <id>typesafe-repo</id>
-      <name>Typesafe Repository</name>
-      <url>http://repo.typesafe.com/typesafe/releases/</url>
+      <id>akka-repo</id>
+      <name>Akka Repository</name>
+      <url>http://repo.akka.io/releases/</url>
       <releases>
         <enabled>true</enabled>
       </releases>
@@ -514,7 +514,7 @@
           <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-core</artifactId>
-            <version>1.0.3</version>
+            <version>1.0.4</version>
           </dependency>
         </dependencies>
       </dependencyManagement>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index af8b5ba01745b59f2c8c82f3c6c6bdb58a01fa45..5b241cc4ef3167314ac936842f464a17c0511f3e 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -9,8 +9,8 @@ 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"
@@ -35,7 +35,7 @@ object SparkBuild extends Build {
 
   def sharedSettings = Defaults.defaultSettings ++ Seq(
     organization := "org.spark-project",
-    version := "0.7.0-SNAPSHOT",
+    version := "0.7.1-SNAPSHOT",
     scalaVersion := "2.9.2",
     scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue
     unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath },
@@ -44,6 +44,9 @@ object SparkBuild extends Build {
     transitiveClassifiers in Scope.GlobalScope := Seq("sources"),
     testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))),
 
+    // 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/"),
@@ -114,7 +117,6 @@ 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/",
@@ -155,9 +157,7 @@ object SparkBuild extends Build {
 
   def examplesSettings = sharedSettings ++ Seq(
     name := "spark-examples",
-    libraryDependencies ++= Seq(
-      "org.twitter4j" % "twitter4j-stream" % "3.0.3"
-    )
+    libraryDependencies ++= Seq("com.twitter" % "algebird-core_2.9.2" % "0.1.8")
   )
 
   def bagelSettings = sharedSettings ++ Seq(name := "spark-bagel")
@@ -166,7 +166,9 @@ object SparkBuild extends Build {
     name := "spark-streaming",
     libraryDependencies ++= Seq(
       "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile",
-      "com.github.sgroschupf" % "zkclient" % "0.1"
+      "com.github.sgroschupf" % "zkclient" % "0.1",
+      "org.twitter4j" % "twitter4j-stream" % "3.0.3",
+      "com.typesafe.akka" % "akka-zeromq" % "2.0.3"
     )
   ) ++ assemblySettings ++ extraAssemblySettings
 
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 6b6ab6abd97e99b8850ac0764ba2c4ebb4cef5d8..172ed85fab9267b7cfffbec8c0474701c9faf23f 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -215,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.
 
@@ -224,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={}):
         """
@@ -274,7 +274,7 @@ class RDD(object):
 
     def reduce(self, f):
         """
-        Reduces the elements of this RDD using the specified commutative and 
+        Reduces the elements of this RDD using the specified commutative and
         associative binary operator.
 
         >>> from operator import add
@@ -422,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):
         """
@@ -474,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}.
@@ -489,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}.
 
@@ -506,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}.
 
@@ -523,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, partitionFunc=hash):
+    def partitionBy(self, numPartitions, partitionFunc=hash):
         """
         Return a copy of the RDD partitioned using the specified partitioner.
 
@@ -535,22 +535,22 @@ 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[partitionFunc(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.PythonPartitioner(numSplits,
+        partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
                                                      id(partitionFunc))
         jrdd = pairRDD.partitionBy(partitioner).values()
         rdd = RDD(jrdd, self.ctx)
@@ -561,7 +561,7 @@ class RDD(object):
 
     # 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.
@@ -586,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:
@@ -597,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:
@@ -609,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())
@@ -630,7 +630,7 @@ class RDD(object):
             return a + b
 
         return self.combineByKey(createCombiner, mergeValue, mergeCombiners,
-                numSplits)
+                numPartitions)
 
     # TODO: add tests
     def flatMapValues(self, f):
@@ -659,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
@@ -670,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
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index 0667b71cc7bba49a22a2c12d7f9c590dc30ea78c..a60028bb53d71220e00cd719522bfd345735854d 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.spark-project</groupId>
     <artifactId>parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.7.1-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
diff --git a/repl/pom.xml b/repl/pom.xml
index 4a296fa630106140250aa7869dfca1a3e6a5d263..a1b3ccece8b3114c2c982c7b49d3fd985069fc8f 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.spark-project</groupId>
     <artifactId>parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.7.1-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala
index 22bcb4be8a8f8b37c9a9c2a73b6565eebc300ce9..cd7b5128b24df21611df32598db2d2cc95b74b03 100644
--- a/repl/src/main/scala/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/spark/repl/SparkILoop.scala
@@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
       ____              __  
      / __/__  ___ _____/ /__
     _\ \/ _ \/ _ `/ __/  '_/
-   /___/ .__/\_,_/_/ /_/\_\   version 0.7.0
+   /___/ .__/\_,_/_/ /_/\_\   version 0.7.1
       /_/                  
 """)
     import Properties._
diff --git a/run b/run
index 82b1da005a44741ab94c20b37ef1b10da3059011..2c29cc4a6641cd6f11f80f5485b34ab27528006e 100755
--- a/run
+++ b/run
@@ -25,6 +25,26 @@ if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker"
   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"
@@ -91,11 +111,13 @@ 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/*"
@@ -112,6 +134,17 @@ for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
 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 c913a5195ef95f83845dffcd1e6cdc497b9747c8..cb20a4b7a2f9a228cf66febbba429e5d39ecb60b 100644
--- a/run2.cmd
+++ b/run2.cmd
@@ -47,11 +47,14 @@ 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
 set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\jars\*
 set CLASSPATH=%CLASSPATH%;%FWDIR%lib_managed\bundles\*
@@ -59,6 +62,16 @@ 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/streaming/pom.xml b/streaming/pom.xml
index 6ee7e59df39d16b1bf47ad23604bfe7f80f40f53..d1a766aeacf454d6966f46626ef550597f9fb9d3 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.spark-project</groupId>
     <artifactId>parent</artifactId>
-    <version>0.7.0-SNAPSHOT</version>
+    <version>0.7.1-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 
@@ -47,7 +47,16 @@
       <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>
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
index 80244520a372642ac5b3f2817e15cf8d879e4238..e303e33e5e4014e7b252b491edc9d5090dc5e88f 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -6,6 +6,8 @@ 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]
@@ -15,9 +17,11 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
   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: Duration = ssc.checkpointDuration
+  val checkpointDuration = ssc.checkpointDuration
+  val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
 
   def validate() {
     assert(master != null, "Checkpoint.master is null")
@@ -37,32 +41,50 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
   val conf = new Configuration()
   var fs = file.getFileSystem(conf)
   val maxAttempts = 3
+  val executor = Executors.newFixedThreadPool(1)
 
-  def write(checkpoint: Checkpoint) {
-    // TODO: maybe do this in a different thread from the main stream execution thread
-    var attempts = 0
-    while (attempts < maxAttempts) {
-      attempts += 1
-      try {
-        logDebug("Saving checkpoint for time " + checkpoint.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)
+  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)
         }
-        val fos = fs.create(file)
-        val oos = new ObjectOutputStream(fos)
-        oos.writeObject(checkpoint)
-        oos.close()
-        logInfo("Checkpoint for time " + checkpoint.checkpointTime + " saved to file '" + file + "'")
-        fos.close()
-        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 + "'")
     }
-    logError("Could not write checkpoint for time " + checkpoint.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()
   }
 }
 
@@ -84,7 +106,8 @@ object CheckpointReader extends Logging {
           // 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 ois = new ObjectInputStreamWithLoader(fis, Thread.currentThread().getContextClassLoader)
+          val zis = new LZFInputStream(fis)
+          val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader)
           val cp = ois.readObject.asInstanceOf[Checkpoint]
           ois.close()
           fs.close()
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala
index 352f83fe0c33bdfee1093efd90aec6206ad825c4..e1be5ef51cc9cd8cc4c71c52b6f8dcb820dbb761 100644
--- a/streaming/src/main/scala/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/spark/streaming/DStream.scala
@@ -12,7 +12,7 @@ import scala.collection.mutable.HashMap
 
 import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
 
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hadoop.conf.Configuration
 
 /**
@@ -75,7 +75,7 @@ abstract class DStream[T: ClassManifest] (
   // Checkpoint details
   protected[streaming] val mustCheckpoint = false
   protected[streaming] var checkpointDuration: Duration = null
-  protected[streaming] var checkpointData = new DStreamCheckpointData(HashMap[Time, Any]())
+  protected[streaming] val checkpointData = new DStreamCheckpointData(this)
 
   // Reference to whole DStream graph
   protected[streaming] var graph: DStreamGraph = null
@@ -85,10 +85,10 @@ abstract class DStream[T: ClassManifest] (
   // Duration for which the DStream requires its parent DStream to remember each RDD created
   protected[streaming] def parentRememberDuration = rememberDuration
 
-  /** Returns the StreamingContext associated with this DStream */
-  def context() = ssc
+  /** Return the StreamingContext associated with this DStream */
+  def context = ssc
 
-  /** Persists the RDDs of this DStream with the given storage level */
+  /** Persist the RDDs of this DStream with the given storage level */
   def persist(level: StorageLevel): DStream[T] = {
     if (this.isInitialized) {
       throw new UnsupportedOperationException(
@@ -132,7 +132,7 @@ abstract class DStream[T: ClassManifest] (
 
     // Set the checkpoint interval to be slideDuration or 10 seconds, which ever is larger
     if (mustCheckpoint && checkpointDuration == null) {
-      checkpointDuration = slideDuration.max(Seconds(10))
+      checkpointDuration = slideDuration * math.ceil(Seconds(10) / slideDuration).toInt
       logInfo("Checkpoint interval automatically set to " + checkpointDuration)
     }
 
@@ -159,7 +159,7 @@ abstract class DStream[T: ClassManifest] (
     )
 
     assert(
-     checkpointDuration == null || ssc.sc.checkpointDir.isDefined,
+     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."
     )
@@ -238,13 +238,15 @@ abstract class DStream[T: ClassManifest] (
     dependencies.foreach(_.remember(parentRememberDuration))
   }
 
-  /** This method checks whether the 'time' is valid wrt slideDuration for generating RDD */
+  /** 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
     }
   }
@@ -292,14 +294,14 @@ abstract class DStream[T: ClassManifest] (
    * 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
-   * (eg. ForEachDStream).
+   * 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]) => {} } 
-          ssc.sc.runJob(rdd, emptyFunc)
+          val emptyFunc = { (iterator: Iterator[T]) => {} }
+          context.sparkContext.runJob(rdd, emptyFunc)
         }
         Some(new Job(time, jobFunc))
       }
@@ -308,20 +310,18 @@ abstract class DStream[T: ClassManifest] (
   }
 
   /**
-   * Dereference RDDs that are older than rememberDuration.
+   * 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 forgetOldRDDs(time: Time) {
-    val keys = generatedRDDs.keys
+  protected[streaming] def clearOldMetadata(time: Time) {
     var numForgotten = 0
-    keys.foreach(t => {
-      if (t <= (time - rememberDuration)) {
-        generatedRDDs.remove(t)
-        numForgotten += 1
-        logInfo("Forgot RDD of time " + t + " from " + this)
-      }
-    })
-    logInfo("Forgot " + numForgotten + " RDDs from " + this)
-    dependencies.foreach(_.forgetOldRDDs(time))
+    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. 
@@ -342,40 +342,10 @@ abstract class DStream[T: ClassManifest] (
    */
   protected[streaming] def updateCheckpointData(currentTime: Time) {
     logInfo("Updating checkpoint data for time " + currentTime)
-
-    // Get the checkpointed RDDs from the generated RDDs
-    val newRdds = generatedRDDs.filter(_._2.getCheckpointFile.isDefined)
-                               .map(x => (x._1, x._2.getCheckpointFile.get))
-
-    // Make a copy of the existing checkpoint data (checkpointed RDDs)
-    val oldRdds = checkpointData.rdds.clone()
-
-    // If the new checkpoint data has checkpoints then replace existing with the new one
-    if (newRdds.size > 0) {
-      checkpointData.rdds.clear()
-      checkpointData.rdds ++= newRdds
-    }
-
-    // Make parent DStreams update their checkpoint data
+    checkpointData.update()
     dependencies.foreach(_.updateCheckpointData(currentTime))
-
-    // TODO: remove this, this is just for debugging
-    newRdds.foreach {
-      case (time, data) => { logInfo("Added checkpointed RDD for time " + time + " to stream checkpoint") }
-    }
-
-    if (newRdds.size > 0) {
-      (oldRdds -- newRdds.keySet).foreach {
-        case (time, data) => {
-          val path = new Path(data.toString)
-          val fs = path.getFileSystem(new Configuration())
-          fs.delete(path, true)
-          logInfo("Deleted checkpoint file '" + path + "' for time " + time)
-        }
-      }
-    }
-    logInfo("Updated checkpoint data for time " + currentTime + ", " + checkpointData.rdds.size + " checkpoints, " 
-      + "[" + checkpointData.rdds.mkString(",") + "]")
+    checkpointData.cleanup()
+    logDebug("Updated checkpoint data for time " + currentTime + ": " + checkpointData)
   }
 
   /**
@@ -386,14 +356,8 @@ abstract class DStream[T: ClassManifest] (
    */
   protected[streaming] def restoreCheckpointData() {
     // Create RDDs from the checkpoint data
-    logInfo("Restoring checkpoint data from " + checkpointData.rdds.size + " checkpointed RDDs")
-    checkpointData.rdds.foreach {
-      case(time, data) => {
-        logInfo("Restoring checkpointed RDD for time " + time + " from file '" + data.toString + "'")
-        val rdd = ssc.sc.checkpointFile[T](data.toString)
-        generatedRDDs += ((time, rdd))
-      }
-    }
+    logInfo("Restoring checkpoint data")
+    checkpointData.restore()
     dependencies.foreach(_.restoreCheckpointData())
     logInfo("Restored checkpoint data")
   }
@@ -433,7 +397,7 @@ abstract class DStream[T: ClassManifest] (
 
   /** 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, ssc.sc.clean(mapFunc))
+    new MappedDStream(this, context.sparkContext.clean(mapFunc))
   }
 
   /**
@@ -441,7 +405,7 @@ abstract class DStream[T: ClassManifest] (
    * and then flattening the results
    */
   def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = {
-    new FlatMappedDStream(this, ssc.sc.clean(flatMapFunc))
+    new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc))
   }
 
   /** Return a new DStream containing only the elements that satisfy a predicate. */
@@ -463,7 +427,7 @@ abstract class DStream[T: ClassManifest] (
       mapPartFunc: Iterator[T] => Iterator[U],
       preservePartitioning: Boolean = false
     ): DStream[U] = {
-    new MapPartitionedDStream(this, ssc.sc.clean(mapPartFunc), preservePartitioning)
+    new MapPartitionedDStream(this, context.sparkContext.clean(mapPartFunc), preservePartitioning)
   }
 
   /**
@@ -479,6 +443,15 @@ abstract class DStream[T: ClassManifest] (
    */
   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.
@@ -492,7 +465,7 @@ abstract class DStream[T: ClassManifest] (
    * this DStream will be registered as an output stream and therefore materialized.
    */
   def foreach(foreachFunc: (RDD[T], Time) => Unit) {
-    val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc))
+    val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
     ssc.registerOutputStream(newStream)
     newStream
   }
@@ -510,7 +483,7 @@ abstract class DStream[T: ClassManifest] (
    * on each RDD of this DStream.
    */
   def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
-    new TransformedDStream(this, ssc.sc.clean(transformFunc))
+    new TransformedDStream(this, context.sparkContext.clean(transformFunc))
   }
 
   /**
@@ -527,19 +500,21 @@ abstract class DStream[T: ClassManifest] (
       if (first11.size > 10) println("...")
       println()
     }
-    val newStream = new ForEachDStream(this, ssc.sc.clean(foreachFunc))
+    val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
     ssc.registerOutputStream(newStream)
   }
 
   /**
-   * 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.
+   * 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 which is computed based on windowed batches of this DStream.
+   * 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
@@ -550,28 +525,40 @@ abstract class DStream[T: ClassManifest] (
     new WindowedDStream(this, windowDuration, slideDuration)
   }
 
-  /**
-   * Return a new DStream which computed based on tumbling window on this DStream.
-   * This is equivalent to window(batchTime, batchTime).
-   * @param batchDuration tumbling window duration; must be a multiple of this DStream's
-   *                  batching interval
-   */
-  def tumble(batchDuration: Duration): DStream[T] = window(batchDuration, batchDuration)
-
   /**
    * Return a new DStream in which each RDD has a single element generated by reducing all
-   * elements in a window over this DStream. windowDuration and slideDuration are as defined
-   * in the window() operation. This is equivalent to
-   * window(windowDuration, slideDuration).reduce(reduceFunc)
+   * 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.window(windowDuration, slideDuration).reduce(reduceFunc)
+    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,
@@ -585,13 +572,46 @@ abstract class DStream[T: ClassManifest] (
 
   /**
    * 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()
+   * 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.
@@ -609,16 +629,21 @@ abstract class DStream[T: ClassManifest] (
    * Return all the RDDs between 'fromTime' to 'toTime' (both included)
    */
   def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = {
-    val rdds = new ArrayBuffer[RDD[T]]()
-    var time = toTime.floor(slideDuration)
-    while (time >= zeroTime && time >= fromTime) {
-      getOrCompute(time) match {
-        case Some(rdd) => rdds += rdd
-        case None => //throw new Exception("Could not get RDD for time " + time)
-      }
-      time -= slideDuration
+    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 + ")")
     }
-    rdds.toSeq
+    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
+    })
   }
 
   /**
@@ -651,7 +676,3 @@ abstract class DStream[T: ClassManifest] (
     ssc.registerOutputStream(this)
   }
 }
-
-private[streaming]
-case class DStreamCheckpointData(rdds: HashMap[Time, Any])
-
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
index bc4a40d7bccd4cd1b277e48119bb9a677a9d999d..adb7f3a24d25f6fcbd1453c2e75b56b9a22d10b4 100644
--- a/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/spark/streaming/DStreamGraph.scala
@@ -11,17 +11,20 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
   private val inputStreams = new ArrayBuffer[InputDStream[_]]()
   private val outputStreams = new ArrayBuffer[DStream[_]]()
 
-  private[streaming] var zeroTime: Time = null
-  private[streaming] var batchDuration: Duration = null
-  private[streaming] var rememberDuration: Duration = null
-  private[streaming] var checkpointInProgress = false
+  var rememberDuration: Duration = null
+  var checkpointInProgress = false
 
-  private[streaming] def start(time: Time) {
+  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)
@@ -29,19 +32,23 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
     }
   }
 
-  private[streaming] def stop() {
+  def restart(time: Time) {
+    this.synchronized { startTime = time }
+  }
+
+  def stop() {
     this.synchronized {
       inputStreams.par.foreach(_.stop())
     }
   }
 
-  private[streaming] def setContext(ssc: StreamingContext) {
+  def setContext(ssc: StreamingContext) {
     this.synchronized {
       outputStreams.foreach(_.setContext(ssc))
     }
   }
 
-  private[streaming] def setBatchDuration(duration: Duration) {
+  def setBatchDuration(duration: Duration) {
     this.synchronized {
       if (batchDuration != null) {
         throw new Exception("Batch duration already set as " + batchDuration +
@@ -51,59 +58,68 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
     batchDuration = duration
   }
 
-  private[streaming] def remember(duration: 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
     }
-    rememberDuration = duration
   }
 
-  private[streaming] def addInputStream(inputStream: InputDStream[_]) {
+  def addInputStream(inputStream: InputDStream[_]) {
     this.synchronized {
       inputStream.setGraph(this)
       inputStreams += inputStream
     }
   }
 
-  private[streaming] def addOutputStream(outputStream: DStream[_]) {
+  def addOutputStream(outputStream: DStream[_]) {
     this.synchronized {
       outputStream.setGraph(this)
       outputStreams += outputStream
     }
   }
 
-  private[streaming] def getInputStreams() = this.synchronized { inputStreams.toArray }
+  def getInputStreams() = this.synchronized { inputStreams.toArray }
 
-  private[streaming] def getOutputStreams() = this.synchronized { outputStreams.toArray }
+  def getOutputStreams() = this.synchronized { outputStreams.toArray }
 
-  private[streaming] def generateRDDs(time: Time): Seq[Job] = {
+  def generateJobs(time: Time): Seq[Job] = {
     this.synchronized {
-      outputStreams.flatMap(outputStream => outputStream.generateJob(time))
+      logInfo("Generating jobs for time " + time)
+      val jobs = outputStreams.flatMap(outputStream => outputStream.generateJob(time))
+      logInfo("Generated " + jobs.length + " jobs for time " + time)
+      jobs
     }
   }
 
-  private[streaming] def forgetOldRDDs(time: Time) {
+  def clearOldMetadata(time: Time) {
     this.synchronized {
-      outputStreams.foreach(_.forgetOldRDDs(time))
+      logInfo("Clearing old metadata for time " + time)
+      outputStreams.foreach(_.clearOldMetadata(time))
+      logInfo("Cleared old metadata for time " + time)
     }
   }
 
-  private[streaming] def updateCheckpointData(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)
     }
   }
 
-  private[streaming] def restoreCheckpointData() {
+  def restoreCheckpointData() {
     this.synchronized {
+      logInfo("Restoring checkpoint data")
       outputStreams.foreach(_.restoreCheckpointData())
+      logInfo("Restored checkpoint data")
     }
   }
 
-  private[streaming] def validate() {
+  def validate() {
     this.synchronized {
       assert(batchDuration != null, "Batch duration has not been set")
       //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + " is very low")
diff --git a/streaming/src/main/scala/spark/streaming/Duration.scala b/streaming/src/main/scala/spark/streaming/Duration.scala
index e4dc579a170a4214ffa3a9ee959b160471a26dbe..ee26206e249a8abf6bf9f502b7dac6eaef65e0e6 100644
--- a/streaming/src/main/scala/spark/streaming/Duration.scala
+++ b/streaming/src/main/scala/spark/streaming/Duration.scala
@@ -16,7 +16,7 @@ case class Duration (private val millis: Long) {
 
   def * (times: Int): Duration = new Duration(millis * times)
 
-  def / (that: Duration): Long = millis / that.millis
+  def / (that: Duration): Double = millis.toDouble / that.millis.toDouble
 
   def isMultipleOf(that: Duration): Boolean =
     (this.millis % that.millis == 0)
diff --git a/streaming/src/main/scala/spark/streaming/Interval.scala b/streaming/src/main/scala/spark/streaming/Interval.scala
index dc21dfb72228801bb1db803901ec81e6b76ae411..6a8b81760e35b5fe81dc169155d7678c107bba9f 100644
--- a/streaming/src/main/scala/spark/streaming/Interval.scala
+++ b/streaming/src/main/scala/spark/streaming/Interval.scala
@@ -30,6 +30,7 @@ class Interval(val beginTime: Time, val endTime: Time) {
   override def toString = "[" + beginTime + ", " + endTime + "]"
 }
 
+private[streaming]
 object Interval {
   def currentInterval(duration: Duration): Interval  = {
     val time = new Time(System.currentTimeMillis)
diff --git a/streaming/src/main/scala/spark/streaming/JobManager.scala b/streaming/src/main/scala/spark/streaming/JobManager.scala
index 3b910538e028925bd9125db6ab4e2045717a2c9e..7696c4a592bf6c3c9f5885da1e779f39f7ff3101 100644
--- a/streaming/src/main/scala/spark/streaming/JobManager.scala
+++ b/streaming/src/main/scala/spark/streaming/JobManager.scala
@@ -3,6 +3,8 @@ package spark.streaming
 import spark.Logging 
 import spark.SparkEnv
 import java.util.concurrent.Executors
+import collection.mutable.HashMap
+import collection.mutable.ArrayBuffer
 
 
 private[streaming]
@@ -13,21 +15,57 @@ class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging {
       SparkEnv.set(ssc.env)
       try {
         val timeTaken = job.run()
-        logInfo("Total delay: %.5f s for job %s (execution: %.5f s)".format(
-          (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, timeTaken / 1000.0))
+        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
index e4152f3a61bb5e3240b38fd706a6e784686152cd..b159d26c02b2d4383253110d1d589fc2a6b4261b 100644
--- a/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/spark/streaming/NetworkInputTracker.scala
@@ -4,6 +4,7 @@ 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
@@ -23,7 +24,7 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) ext
  */
 private[streaming]
 class NetworkInputTracker(
-    @transient ssc: StreamingContext, 
+    @transient ssc: StreamingContext,
     @transient networkInputStreams: Array[NetworkInputDStream[_]])
   extends Logging {
 
@@ -65,12 +66,12 @@ class NetworkInputTracker(
     def receive = {
       case RegisterReceiver(streamId, receiverActor) => {
         if (!networkInputStreamMap.contains(streamId)) {
-          throw new Exception("Register received for unexpected id " + 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)) {
@@ -85,7 +86,7 @@ class NetworkInputTracker(
       }
       case DeregisterReceiver(streamId, msg) => {
         receiverInfo -= streamId
-        logInfo("De-registered receiver for network stream " + streamId
+        logError("De-registered receiver for network stream " + streamId
           + " with message " + msg)
         //TODO: Do something about the corresponding NetworkInputDStream
       }
@@ -95,8 +96,8 @@ class NetworkInputTracker(
   /** This thread class runs all the receivers on the cluster.  */
   class ReceiverExecutor extends Thread {
     val env = ssc.env
-        
-    override def run() {      
+
+    override def run() {
       try {
         SparkEnv.set(env)
         startReceivers()
@@ -113,7 +114,7 @@ class NetworkInputTracker(
      */
     def startReceivers() {
       val receivers = networkInputStreams.map(nis => {
-        val rcvr = nis.createReceiver()
+        val rcvr = nis.getReceiver()
         rcvr.setStreamId(nis.id)
         rcvr
       })
@@ -138,10 +139,14 @@ class NetworkInputTracker(
         }
         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.sc.runJob(tempRDD, startReceiver)
+      ssc.sparkContext.runJob(tempRDD, startReceiver)
     }
-    
+
     /** Stops the receivers. */
     def stopReceivers() {
       // Signal the receivers to stop
diff --git a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
index 5db3844f1d21e097ebc8cf1bd6cd62e25aa3abba..3ec922957d635135f21f279005405ec556c9d898 100644
--- a/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/spark/streaming/PairDStreamFunctions.scala
@@ -18,15 +18,15 @@ import org.apache.hadoop.conf.Configuration
 
 class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)])
 extends Serializable {
- 
-  def ssc = self.ssc
+
+  private[streaming] def ssc = self.ssc
 
   private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = {
     new HashPartitioner(numPartitions)
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * 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])] = {
@@ -34,7 +34,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * 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])] = {
@@ -42,7 +42,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` on each RDD. The supplied [[spark.Partitioner]]
+   * 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])] = {
@@ -54,7 +54,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * 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.
    */
@@ -63,7 +63,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * 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.
    */
@@ -72,7 +72,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * 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.
    */
@@ -82,7 +82,7 @@ extends Serializable {
   }
 
   /**
-   * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+   * 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.
    */
@@ -95,15 +95,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by counting the number of values of each key in each RDD. Hash
-   * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions.
-   */
-  def countByKey(numPartitions: Int = self.ssc.sc.defaultParallelism): DStream[(K, Long)] = {
-    self.map(x => (x._1, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions)
-  }
-
-  /**
-   * Creates a new DStream by applying `groupByKey` over a sliding window. This is similar to
+   * 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.
@@ -115,7 +107,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` over a sliding window. Similar to
+   * 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
@@ -129,7 +121,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * 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
@@ -137,7 +129,8 @@ extends Serializable {
    * @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 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,
@@ -155,7 +148,7 @@ extends Serializable {
    * @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 partitioner    partitioner for controlling the partitioning of each RDD in the new DStream.
    */
   def groupByKeyAndWindow(
       windowDuration: Duration,
@@ -166,7 +159,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window on `this` DStream.
+   * 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.
@@ -182,7 +175,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * 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
@@ -201,7 +194,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * 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
@@ -210,10 +203,10 @@ extends Serializable {
    * @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 numPartitions  number of partitions of each RDD in the new DStream.
    */
   def reduceByKeyAndWindow(
-      reduceFunc: (V, V) => V, 
+      reduceFunc: (V, V) => V,
       windowDuration: Duration,
       slideDuration: Duration,
       numPartitions: Int
@@ -222,7 +215,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window. Similar to
+   * 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
@@ -230,7 +223,8 @@ extends Serializable {
    * @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 partitioner    partitioner for controlling the partitioning of each RDD
+   *                       in the new DStream.
    */
   def reduceByKeyAndWindow(
       reduceFunc: (V, V) => V,
@@ -245,118 +239,78 @@ extends Serializable {
   }
 
   /**
-   * Create 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 :
+   * 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 that reduceByKeyAndWindow without "inverse reduce" function.
+   *
+   * 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 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
+      slideDuration: Duration = self.slideDuration,
+      numPartitions: Int = ssc.sc.defaultParallelism,
+      filterFunc: ((K, V)) => Boolean = null
     ): DStream[(K, V)] = {
 
     reduceByKeyAndWindow(
-      reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner())
-  }
-
-  /**
-   * Create 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 `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.
-   */
-  def reduceByKeyAndWindow(
-      reduceFunc: (V, V) => V,
-      invReduceFunc: (V, V) => V,
-      windowDuration: Duration,
-      slideDuration: Duration,
-      numPartitions: Int
-    ): DStream[(K, V)] = {
-
-    reduceByKeyAndWindow(
-      reduceFunc, invReduceFunc, windowDuration, slideDuration, defaultPartitioner(numPartitions))
+      reduceFunc, invReduceFunc, windowDuration,
+      slideDuration, defaultPartitioner(numPartitions), filterFunc
+    )
   }
 
   /**
-   * Create 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 :
+   * 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 that reduceByKeyAndWindow without "inverse reduce" function.
+   * 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 function
+   * @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 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
+      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, windowDuration, slideDuration, partitioner)
-  }
-
-  /**
-   * Create a new DStream by counting the number of values for each key over a 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 countByKeyAndWindow(
-      windowDuration: Duration,
-      slideDuration: Duration,
-      numPartitions: Int = self.ssc.sc.defaultParallelism
-    ): DStream[(K, Long)] = {
-
-    self.map(x => (x._1, 1L)).reduceByKeyAndWindow(
-      (x: Long, y: Long) => x + y,
-      (x: Long, y: Long) => x - y,
-      windowDuration,
-      slideDuration,
-      numPartitions
+      self, cleanedReduceFunc, cleanedInvReduceFunc, cleanedFilterFunc,
+      windowDuration, slideDuration, partitioner
     )
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of each key.
    * Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -370,7 +324,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of each key.
    * Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -405,7 +359,7 @@ extends Serializable {
   }
 
   /**
-   * Create a new "state" DStream where the state for each key is updated by applying
+   * Return a new "state" DStream where the state for each key is updated by applying
    * the given function on the previous state of the key and the new values of each key.
    * [[spark.Paxrtitioner]] is used to control the partitioning of each RDD.
    * @param updateFunc State update function. If `this` function returns None, then
@@ -447,7 +401,7 @@ extends Serializable {
   }
 
   /**
-   * Cogroup `this` DStream with `other` DStream. For each key k in corresponding RDDs of `this`
+   * 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.
    */
diff --git a/streaming/src/main/scala/spark/streaming/Scheduler.scala b/streaming/src/main/scala/spark/streaming/Scheduler.scala
index c04ed37de8b21708bdc7ef295441129386f4965b..1c4b22a8981c8db8ecb2a28fcfae2e301d000627 100644
--- a/streaming/src/main/scala/spark/streaming/Scheduler.scala
+++ b/streaming/src/main/scala/spark/streaming/Scheduler.scala
@@ -9,11 +9,8 @@ class Scheduler(ssc: StreamingContext) extends Logging {
 
   initLogging()
 
-  val graph = ssc.graph
-
   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 {
@@ -23,54 +20,93 @@ class Scheduler(ssc: StreamingContext) extends Logging {
   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 => generateRDDs(new Time(longTime)))
+    longTime => generateJobs(new Time(longTime)))
+  val graph = ssc.graph
+  var latestTime: Time = null
 
-  def start() {
-    // If context was started from checkpoint, then restart timer such that
-    // this timer's triggers occur at the same time as the original timer.
-    // Otherwise just start the timer from scratch, and initialize graph based
-    // on this first trigger time of the timer.
+  def start() = synchronized {
     if (ssc.isCheckpointPresent) {
-      // 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.getInitialCheckpoint.checkpointTime.milliseconds
-        val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
-        clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
-      }
-      timer.restart(graph.zeroTime.milliseconds)
-      logInfo("Scheduler's timer restarted")
+      restart()
     } else {
-      val firstTime = new Time(timer.start())
-      graph.start(firstTime - ssc.graph.batchDuration)
-      logInfo("Scheduler's timer started")
+      startFirstTime()
     }
     logInfo("Scheduler started")
   }
   
-  def stop() {
+  def stop() = synchronized {
     timer.stop()
-    graph.stop()
+    jobManager.stop()
+    if (checkpointWriter != null) checkpointWriter.stop()
+    ssc.graph.stop()
     logInfo("Scheduler stopped")    
   }
-  
-  private def generateRDDs(time: Time) {
+
+  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.generateRDDs(time).foreach(jobManager.runJob)
-    graph.forgetOldRDDs(time)
+    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)
-    logInfo("Generated RDDs for time " + time)
   }
 
-  private def doCheckpoint(time: Time) {
+  /** Perform checkpoint for the give `time`. */
+  def doCheckpoint(time: Time) = synchronized {
     if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
-      val startTime = System.currentTimeMillis()
+      logInfo("Checkpointing graph for time " + time)
       ssc.graph.updateCheckpointData(time)
       checkpointWriter.write(new Checkpoint(ssc, time))
-      val stopTime = System.currentTimeMillis()
-      logInfo("Checkpointing the graph took " + (stopTime - startTime) + " ms")
     }
   }
 }
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
index 0cce2b13cf8c45f0162b6e2841e73a6600850a54..b8b60aab43303197a06ae66a84637d23437e7472 100644
--- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
@@ -1,22 +1,32 @@
 package spark.streaming
 
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
+import akka.zeromq.Subscribe
+
 import spark.streaming.dstream._
 
-import spark.{RDD, Logging, SparkEnv, SparkContext}
+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 java.util.UUID
+import twitter4j.Status
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -30,23 +40,34 @@ class StreamingContext private (
   ) extends Logging {
 
   /**
-   * Creates a StreamingContext using an existing SparkContext.
+   * 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)
+  def this(sparkContext: SparkContext, batchDuration: Duration) = {
+    this(sparkContext, null, batchDuration)
+  }
 
   /**
-   * Creates a StreamingContext by providing the details necessary for creating a new SparkContext.
+   * 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) =
-    this(StreamingContext.createNewSparkContext(master, appName), null, batchDuration)
+  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-creates a StreamingContext from a checkpoint file.
+   * 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'.
    */
@@ -55,15 +76,20 @@ class StreamingContext private (
   initLogging()
 
   if (sc_ == null && cp_ == null) {
-    throw new Exception("Streaming Context cannot be initilalized with " +
+    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)
 
-  val sc: SparkContext = {
+  protected[streaming] val sc: SparkContext = {
     if (isCheckpointPresent) {
-      new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars)
+      new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment)
     } else {
       sc_
     }
@@ -101,7 +127,12 @@ class StreamingContext private (
   protected[streaming] var scheduler: Scheduler = null
 
   /**
-   * Sets each DStreams in this context to remember RDDs it generated in the last given duration.
+   * 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).
@@ -112,71 +143,119 @@ class StreamingContext private (
   }
 
   /**
-   * Sets the context to periodically checkpoint the DStream operations for master
-   * fault-tolerance. By default, the graph will be checkpointed every batch interval.
+   * 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
-   * @param interval checkpoint interval
    */
-  def checkpoint(directory: String, interval: Duration = null) {
+  def checkpoint(directory: String) {
     if (directory != null) {
       sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
       checkpointDir = directory
-      checkpointDuration = interval
     } else {
       checkpointDir = null
-      checkpointDuration = null
     }
   }
 
-  protected[streaming] def getInitialCheckpoint(): Checkpoint = {
+  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 hostname Zookeper hostname.
-   * @param port Zookeper port.
+   * @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.
+   * @param storageLevel  Storage level to use for storing the received objects
+   *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
    */
   def kafkaStream[T: ClassManifest](
-      hostname: String,
-      port: Int,
+      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, hostname, port, groupId, topics, initialOffsets, storageLevel)
+    val inputStream = new KafkaInputDStream[T](this, zkQuorum, groupId, topics, initialOffsets, storageLevel)
     registerInputStream(inputStream)
     inputStream
   }
 
   /**
-   * 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
+   * 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 networkTextStream(
+  def socketTextStream(
       hostname: String,
       port: Int,
       storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
     ): DStream[String] = {
-    networkStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
+    socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
   }
 
   /**
-   * Create a input stream from network source hostname:port. Data is received using
+   * 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
@@ -185,7 +264,7 @@ class StreamingContext private (
    * @param storageLevel  Storage level to use for storing the received objects
    * @tparam T            Type of the objects received (after converting bytes to objects)
    */
-  def networkStream[T: ClassManifest](
+  def socketStream[T: ClassManifest](
       hostname: String,
       port: Int,
       converter: (InputStream) => Iterator[T],
@@ -197,7 +276,7 @@ class StreamingContext private (
   }
 
   /**
-   * Creates a input stream from a Flume source.
+   * 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
@@ -222,7 +301,7 @@ class StreamingContext private (
    * @param storageLevel  Storage level to use for storing the received objects
    * @tparam T            Type of the objects in the received blocks
    */
-  def rawNetworkStream[T: ClassManifest](
+  def rawSocketStream[T: ClassManifest](
       hostname: String,
       port: Int,
       storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
@@ -233,7 +312,7 @@ class StreamingContext private (
   }
 
   /**
-   * Creates a input stream that monitors a Hadoop-compatible filesystem
+   * 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
@@ -252,7 +331,7 @@ class StreamingContext private (
   }
 
   /**
-   * Creates a input stream that monitors a Hadoop-compatible filesystem
+   * 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
@@ -271,9 +350,8 @@ class StreamingContext private (
     inputStream
   }
 
-
   /**
-   * Creates a input stream that monitors a Hadoop-compatible filesystem
+   * 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
@@ -283,17 +361,49 @@ class StreamingContext private (
   }
 
   /**
-   * Creates a input stream from an queue of RDDs. In each batch,
+   * 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
+   * @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 = true,
-      defaultRDD: RDD[T] = null
+      oneAtATime: Boolean,
+      defaultRDD: RDD[T]
     ): DStream[T] = {
     val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
     registerInputStream(inputStream)
@@ -308,7 +418,7 @@ class StreamingContext private (
   }
 
   /**
-   * Registers an input stream that will be started (InputDStream.start() called) to get the
+   * Register an input stream that will be started (InputDStream.start() called) to get the
    * input data.
    */
   def registerInputStream(inputStream: InputDStream[_]) {
@@ -316,7 +426,7 @@ class StreamingContext private (
   }
 
   /**
-   * Registers an output stream that will be computed every interval
+   * Register an output stream that will be computed every interval
    */
   def registerOutputStream(outputStream: DStream[_]) {
     graph.addOutputStream(outputStream)
@@ -334,7 +444,7 @@ class StreamingContext private (
   }
 
   /**
-   * Starts the execution of the streams.
+   * Start the execution of the streams.
    */
   def start() {
     if (checkpointDir != null && checkpointDuration == null && graph != null) {
@@ -362,7 +472,7 @@ class StreamingContext private (
   }
 
   /**
-   * Sstops the execution of the streams.
+   * Stop the execution of the streams.
    */
   def stop() {
     try {
@@ -384,14 +494,18 @@ object StreamingContext {
     new PairDStreamFunctions[K, V](stream)
   }
 
-  protected[streaming] def createNewSparkContext(master: String, appName: String): SparkContext = {
-
+  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)
+    new SparkContext(master, appName, sparkHome, jars, environment)
   }
 
   protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
@@ -408,4 +522,3 @@ object StreamingContext {
     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
index 5daeb761ddd670b6daa50ad5226ffb07496e5218..f14decf08ba8f98bc4f59e062b0b8ea17d7ade1e 100644
--- a/streaming/src/main/scala/spark/streaming/Time.scala
+++ b/streaming/src/main/scala/spark/streaming/Time.scala
@@ -37,6 +37,19 @@ case class Time(private val millis: Long) {
 
   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
index 2e7466b16c93edb162ac1027a0a0e75dfff7b850..4d93f0a5f729e48cbda0c18f6104449ccb48abc7 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStream.scala
@@ -4,6 +4,7 @@ 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
@@ -16,9 +17,7 @@ import spark.storage.StorageLevel
  *
  * 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`. 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.
+ * 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
@@ -26,7 +25,9 @@ import spark.storage.StorageLevel
  *  - 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]] {
+    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] =
@@ -36,7 +37,7 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
   def cache(): JavaDStream[T] = dstream.cache()
 
   /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
-  def persist(): JavaDStream[T] = dstream.cache()
+  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)
@@ -50,33 +51,26 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
   }
 
   /**
-   * 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.
+   * 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.
-   * @return
    */
   def window(windowDuration: Duration): JavaDStream[T] =
     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
+   * 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 interval
+   *                       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 which computed based on tumbling window on this DStream.
-   * This is equivalent to window(batchDuration, batchDuration).
-   * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval
-   */
-  def tumble(batchDuration: Duration): JavaDStream[T] =
-    dstream.tumble(batchDuration)
-
   /**
    * 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.
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
index ec546c81907c1e6b203b403b29a7e64496c92113..548809a359644d21785bc4f8c96951a42f1c07f3 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaDStreamLike.scala
@@ -6,17 +6,20 @@ import java.lang.{Long => JLong}
 import scala.collection.JavaConversions._
 
 import spark.streaming._
-import spark.api.java.JavaRDD
+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]] extends Serializable {
+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(_))
   }
@@ -33,6 +36,26 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
    */
   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
@@ -42,6 +65,39 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
     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
@@ -114,8 +170,38 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
 
   /**
    * Return a new DStream in which each RDD has a single element generated by reducing all
-   * elements in a window over this DStream. windowDuration and slideDuration are as defined in the
-   * window() operation. This is equivalent to window(windowDuration, slideDuration).reduce(reduceFunc)
+   * 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],
@@ -129,35 +215,35 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
   /**
    * Return all the RDDs between 'fromDuration' to 'toDuration' (both included)
    */
-  def slice(fromDuration: Duration, toDuration: Duration): JList[JavaRDD[T]] = {
-    new util.ArrayList(dstream.slice(fromDuration, toDuration).map(new JavaRDD(_)).toSeq)
+  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[JavaRDD[T], Void]) {
-    dstream.foreach(rdd => foreachFunc.call(new JavaRDD(rdd)))
+  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[JavaRDD[T], Time, Void]) {
-    dstream.foreach((rdd, time) => foreachFunc.call(new JavaRDD(rdd), time))
+  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[JavaRDD[T], JavaRDD[U]]): JavaDStream[U] = {
+  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(new JavaRDD[T](in)).rdd
+      transformFunc.call(wrapRDD(in)).rdd
     dstream.transform(scalaTransform(_))
   }
 
@@ -165,11 +251,41 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This]] extends Serializable
    * 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[JavaRDD[T], Time, JavaRDD[U]]): JavaDStream[U] = {
+  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(new JavaRDD[T](in), time).rdd
+      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(_, _))
   }
 
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
index eb2495e3acc27b759cd23c6d2706213e61a9d493..30240cad988be32e82c701a0dd7b535156c3dcb4 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaPairDStream.scala
@@ -15,27 +15,30 @@ 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]] {
+    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
   // =======================================================================
 
-  /** Returns a new DStream containing only the elements that satisfy a predicate. */
+  /** 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()))
 
-  /** Persists RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
+  /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
   def cache(): JavaPairDStream[K, V] = dstream.cache()
 
-  /** Persists RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */
-  def persist(): 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()
 
-  /** Persists the RDDs of this DStream with the given storage level */
+  /** 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 */
@@ -67,70 +70,32 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     dstream.window(windowDuration, slideDuration)
 
   /**
-   * Returns a new DStream which computed based on tumbling window on this DStream.
-   * This is equivalent to window(batchDuration, batchDuration).
-   * @param batchDuration tumbling window duration; must be a multiple of this DStream's interval
-   */
-  def tumble(batchDuration: Duration): JavaPairDStream[K, V] =
-    dstream.tumble(batchDuration)
-
-  /**
-   * Returns a new DStream by unifying data of another DStream with this DStream.
+   * 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)
 
-  /**
-   * 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[JavaPairRDD[K, V], 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[(K, V)]): RDD[(K2, V2)] =
-      transformFunc.call(new JavaPairRDD[K, V](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[JavaPairRDD[K, V], 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[(K, V)], time: Time): RDD[(K2, V2)] =
-      transformFunc.call(new JavaPairRDD[K, V](in), time).rdd
-    dstream.transform(scalaTransform(_, _))
-  }
-
   // =======================================================================
   // Methods only for PairDStream's
   // =======================================================================
 
   /**
-   * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * 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 _)
 
   /**
-   * Create a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to
+   * 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 _)
 
   /**
-   * Creates a new DStream by applying `groupByKey` on each RDD of `this` DStream.
+   * 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.
@@ -139,7 +104,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     dstream.groupByKey(partitioner).mapValues(seqAsJavaList _)
 
   /**
-   * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * 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.
    */
@@ -147,7 +112,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     dstream.reduceByKey(func)
 
   /**
-   * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * 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.
    */
@@ -155,7 +120,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
     dstream.reduceByKey(func, numPartitions)
 
   /**
-   * Create a new DStream by applying `reduceByKey` to each RDD. The values for each key are
+   * 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.
    */
@@ -179,24 +144,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by counting the number of values of each key in each RDD. Hash
-   * partitioning is used to generate the RDDs with Spark's `numPartitions` partitions.
-   */
-  def countByKey(numPartitions: Int): JavaPairDStream[K, JLong] = {
-    JavaPairDStream.scalaToJavaLong(dstream.countByKey(numPartitions));
-  }
-
-
-  /**
-   * Create a new DStream by counting the number of values of each key in each RDD. Hash
-   * partitioning is used to generate the RDDs with the default number of partitions.
-   */
-  def countByKey(): JavaPairDStream[K, JLong] = {
-    JavaPairDStream.scalaToJavaLong(dstream.countByKey());
-  }
-
-  /**
-   * Creates a new DStream by applying `groupByKey` over a sliding window. This is similar to
+   * 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.
@@ -208,7 +156,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` over a sliding window. Similar to
+   * 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
@@ -223,7 +171,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * 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
@@ -240,7 +188,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by applying `groupByKey` over a sliding window on `this` DStream.
+   * 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
@@ -273,7 +221,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * 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
@@ -292,7 +240,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window. This is similar to
+   * 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
@@ -313,7 +261,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by applying `reduceByKey` over a sliding window. Similar to
+   * 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
@@ -333,7 +281,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by reducing over a using incremental computation.
+   * 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)
@@ -358,7 +306,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
   }
 
   /**
-   * Create a new DStream by reducing over a using incremental computation.
+   * 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)
@@ -372,25 +320,31 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
    * @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 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
+      numPartitions: Int,
+      filterFunc: JFunction[(K, V), java.lang.Boolean]
     ): JavaPairDStream[K, V] = {
     dstream.reduceByKeyAndWindow(
         reduceFunc,
         invReduceFunc,
         windowDuration,
         slideDuration,
-        numPartitions)
+        numPartitions,
+        (p: (K, V)) => filterFunc(p).booleanValue()
+    )
   }
 
   /**
-   * Create a new DStream by reducing over a using incremental computation.
+   * 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)
@@ -404,49 +358,26 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
    *                       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
-    ): JavaPairDStream[K, V] = {
+      partitioner: Partitioner,
+      filterFunc: JFunction[(K, V), java.lang.Boolean]
+  ): JavaPairDStream[K, V] = {
     dstream.reduceByKeyAndWindow(
         reduceFunc,
         invReduceFunc,
         windowDuration,
         slideDuration,
-        partitioner)
-  }
-
-  /**
-   * Create a new DStream by counting the number of values for each key over a 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
-   */
-  def countByKeyAndWindow(windowDuration: Duration, slideDuration: Duration)
-  : JavaPairDStream[K, JLong] = {
-    JavaPairDStream.scalaToJavaLong(dstream.countByKeyAndWindow(windowDuration, slideDuration))
-  }
-
-  /**
-   * Create a new DStream by counting the number of values for each key over a 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 countByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int)
-   : JavaPairDStream[K, Long] = {
-    dstream.countByKeyAndWindow(windowDuration, slideDuration, numPartitions)
+        partitioner,
+        (p: (K, V)) => filterFunc(p).booleanValue()
+    )
   }
 
   private def convertUpdateStateFunction[S](in: JFunction2[JList[V], Optional[S], Optional[S]]):
diff --git a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
index e5b5e9ac233ed00a089db0ea2b87e300071e9f65..3d149a742cb3cf37de71c26d8e6aa77bdb7de408 100644
--- a/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/api/java/JavaStreamingContext.scala
@@ -1,16 +1,26 @@
 package spark.streaming.api.java
 
-import scala.collection.JavaConversions._
-import java.lang.{Long => JLong, Integer => JInt}
-
 import spark.streaming._
-import dstream._
+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}
-import spark.api.java.{JavaSparkContext, JavaRDD}
 
 /**
  * A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -31,10 +41,63 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @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))
+    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
    */
@@ -53,27 +116,24 @@ class JavaStreamingContext(val ssc: StreamingContext) {
 
   /**
    * Create an input stream that pulls messages form a Kafka Broker.
-   * @param hostname Zookeper hostname.
-   * @param port Zookeper port.
+   * @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](
-    hostname: String,
-    port: Int,
+    zkQuorum: String,
     groupId: String,
     topics: JMap[String, JInt])
   : JavaDStream[T] = {
     implicit val cmt: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    ssc.kafkaStream[T](hostname, port, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+    ssc.kafkaStream[T](zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
   }
 
   /**
    * Create an input stream that pulls messages form a Kafka Broker.
-   * @param hostname Zookeper hostname.
-   * @param port Zookeper port.
+   * @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.
@@ -81,8 +141,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * By default the value is pulled from zookeper.
    */
   def kafkaStream[T](
-    hostname: String,
-    port: Int,
+    zkQuorum: String,
     groupId: String,
     topics: JMap[String, JInt],
     initialOffsets: JMap[KafkaPartitionKey, JLong])
@@ -90,8 +149,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     implicit val cmt: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
     ssc.kafkaStream[T](
-      hostname,
-      port,
+      zkQuorum,
       groupId,
       Map(topics.mapValues(_.intValue()).toSeq: _*),
       Map(initialOffsets.mapValues(_.longValue()).toSeq: _*))
@@ -99,8 +157,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
 
   /**
    * Create an input stream that pulls messages form a Kafka Broker.
-   * @param hostname Zookeper hostname.
-   * @param port Zookeper port.
+   * @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.
@@ -109,8 +166,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param storageLevel RDD storage level. Defaults to memory-only
    */
   def kafkaStream[T](
-    hostname: String,
-    port: Int,
+    zkQuorum: String,
     groupId: String,
     topics: JMap[String, JInt],
     initialOffsets: JMap[KafkaPartitionKey, JLong],
@@ -119,8 +175,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     implicit val cmt: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
     ssc.kafkaStream[T](
-      hostname,
-      port,
+      zkQuorum,
       groupId,
       Map(topics.mapValues(_.intValue()).toSeq: _*),
       Map(initialOffsets.mapValues(_.longValue()).toSeq: _*),
@@ -136,9 +191,9 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param storageLevel  Storage level to use for storing the received objects
    *                      (default: StorageLevel.MEMORY_AND_DISK_SER_2)
    */
-  def networkTextStream(hostname: String, port: Int, storageLevel: StorageLevel)
+  def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel)
   : JavaDStream[String] = {
-    ssc.networkTextStream(hostname, port, storageLevel)
+    ssc.socketTextStream(hostname, port, storageLevel)
   }
 
   /**
@@ -148,8 +203,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param hostname      Hostname to connect to for receiving data
    * @param port          Port to connect to for receiving data
    */
-  def networkTextStream(hostname: String, port: Int): JavaDStream[String] = {
-    ssc.networkTextStream(hostname, port)
+  def socketTextStream(hostname: String, port: Int): JavaDStream[String] = {
+    ssc.socketTextStream(hostname, port)
   }
 
   /**
@@ -162,7 +217,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param storageLevel  Storage level to use for storing the received objects
    * @tparam T            Type of the objects received (after converting bytes to objects)
    */
-  def networkStream[T](
+  def socketStream[T](
       hostname: String,
       port: Int,
       converter: JFunction[InputStream, java.lang.Iterable[T]],
@@ -171,7 +226,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
     def fn = (x: InputStream) => converter.apply(x).toIterator
     implicit val cmt: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    ssc.networkStream(hostname, port, fn, storageLevel)
+    ssc.socketStream(hostname, port, fn, storageLevel)
   }
 
   /**
@@ -194,13 +249,13 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param storageLevel  Storage level to use for storing the received objects
    * @tparam T            Type of the objects in the received blocks
    */
-  def rawNetworkStream[T](
+  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.rawNetworkStream(hostname, port, storageLevel))
+    JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel))
   }
 
   /**
@@ -212,10 +267,10 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @param port          Port to connect to for receiving data
    * @tparam T            Type of the objects in the received blocks
    */
-  def rawNetworkStream[T](hostname: String, port: Int): JavaDStream[T] = {
+  def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = {
     implicit val cmt: ClassManifest[T] =
       implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
-    JavaDStream.fromDStream(ssc.rawNetworkStream(hostname, port))
+    JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port))
   }
 
   /**
@@ -254,15 +309,182 @@ class JavaStreamingContext(val ssc: StreamingContext) {
    * @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] = {
+  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[_, _]) {
+  def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) {
     ssc.registerOutputStream(outputStream.dstream)
   }
 
@@ -322,12 +544,11 @@ class JavaStreamingContext(val ssc: StreamingContext) {
 
   /**
    * Sets the context to periodically checkpoint the DStream operations for master
-   * fault-tolerance. By default, the graph will be checkpointed every batch interval.
+   * fault-tolerance. The graph will be checkpointed every batch interval.
    * @param directory HDFS-compatible directory where the checkpoint data will be reliably stored
-   * @param interval checkpoint interval
    */
-  def checkpoint(directory: String, interval: Duration = null) {
-    ssc.checkpoint(directory, interval)
+  def checkpoint(directory: String) {
+    ssc.checkpoint(directory)
   }
 
   /**
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
index 1e6ad84b44014c4d1c063b3b1effed934d532c87..41b9bd9461d9cc5ec93efe43bebdad62de0d9223 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FileInputDStream.scala
@@ -2,13 +2,14 @@ package spark.streaming.dstream
 
 import spark.RDD
 import spark.rdd.UnionRDD
-import spark.streaming.{StreamingContext, Time}
+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
+import scala.collection.mutable.{HashSet, HashMap}
+import java.io.{ObjectInputStream, IOException}
 
 private[streaming]
 class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest](
@@ -18,28 +19,23 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
     newFilesOnly: Boolean = true) 
   extends InputDStream[(K, V)](ssc_) {
 
-  @transient private var path_ : Path = null
-  @transient private var fs_ : FileSystem = null
-
-  var lastModTime = 0L
-  val lastModTimeFiles = new HashSet[String]()
+  protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
 
-  def path(): Path = {
-    if (path_ == null) path_ = new Path(directory)
-    path_
-  }
+  // Latest file mod time seen till any point of time
+  private val lastModTimeFiles = new HashSet[String]()
+  private var lastModTime = 0L
 
-  def fs(): FileSystem = {
-    if (fs_ == null) fs_ = path.getFileSystem(new Configuration())
-    fs_
-  }
+  @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 = System.currentTimeMillis()
+      lastModTime = graph.zeroTime.milliseconds
     } else {
       lastModTime = 0
     }
+    logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly)
   }
   
   override def stop() { }
@@ -49,38 +45,50 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
    * 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. Hence, new files may have the same modification time as the
-   * latest modification time in the previous call to this method and the list of files
-   * maintained is used to filter the one that have been processed.
+   * 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)) {
+        if (!filter(path)) {  // Reject file if it does not satisfy filter
+          logDebug("Rejected by filter " + path)
           return false
-        } else {
+        } else {              // Accept file only if
           val modTime = fs.getFileStatus(path).getModificationTime()
-          if (modTime < lastModTime){
-            return false
+          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)) {
-            return false
+            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
         }        
       }
     }
-
-    val newFiles = fs.listStatus(path, newFilter)
-    logInfo("New files: " + newFiles.map(_.getPath).mkString(", "))
+    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) {
@@ -88,10 +96,81 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
         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]"
     }
-    val newRDD = new UnionRDD(ssc.sc, newFiles.map(
-      file => ssc.sc.newAPIHadoopFile[K, V, F](file.getPath.toString)))
-    Some(newRDD)
   }
 }
 
@@ -100,3 +179,4 @@ object FileInputDStream {
   def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".")
 }
 
+
diff --git a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
index efc70584805697bffa744f82664ff59c9578cbf6..c9644b3a83aa053e96519cab70d5754b6b6803c3 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/FlumeInputDStream.scala
@@ -25,7 +25,7 @@ class FlumeInputDStream[T: ClassManifest](
   storageLevel: StorageLevel
 ) extends NetworkInputDStream[SparkFlumeEvent](ssc_) {
 
-  override def createReceiver(): NetworkReceiver[SparkFlumeEvent] = {
+  override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = {
     new FlumeReceiver(host, port, storageLevel)
   }
 }
@@ -134,4 +134,4 @@ class FlumeReceiver(
   }
 
   override def getLocationPreference = Some(host)
-}
\ No newline at end of file
+}
diff --git a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
index 980ca5177eb1d20a5b8b8a85155e914ff94e1511..3c5d43a60955a38273aabd224073c8fabf2830d4 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/InputDStream.scala
@@ -1,10 +1,42 @@
 package spark.streaming.dstream
 
-import spark.streaming.{Duration, StreamingContext, 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 = {
@@ -13,7 +45,9 @@ abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContex
     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
index 2b4740bdf7c7df14e7fe5cbeea1b1d89096714ae..ddd9becf325e02e649d54d20278e2db37ed795af 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/KafkaInputDStream.scala
@@ -13,27 +13,18 @@ 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)
-// NOT USED - Originally intended for fault-tolerance
-// Metadata for a Kafka Stream that it sent to the Master
-private[streaming]
-case class KafkaInputDStreamMetadata(timestamp: Long, data: Map[KafkaPartitionKey, Long])
-// NOT USED - Originally intended for fault-tolerance
-// Checkpoint data specific to a KafkaInputDstream
-private[streaming]
-case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any],
-  savedOffsets: Map[KafkaPartitionKey, Long]) extends DStreamCheckpointData(kafkaRdds)
 
 /**
  * Input stream that pulls messages from a Kafka Broker.
  * 
- * @param host Zookeper hostname.
- * @param port Zookeper port.
+ * @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.
@@ -44,65 +35,22 @@ case class KafkaDStreamCheckpointData(kafkaRdds: HashMap[Time, Any],
 private[streaming]
 class KafkaInputDStream[T: ClassManifest](
     @transient ssc_ : StreamingContext,
-    host: String,
-    port: Int,
+    zkQuorum: String,
     groupId: String,
     topics: Map[String, Int],
     initialOffsets: Map[KafkaPartitionKey, Long],
     storageLevel: StorageLevel
   ) extends NetworkInputDStream[T](ssc_ ) with Logging {
 
-  // Metadata that keeps track of which messages have already been consumed.
-  var savedOffsets = HashMap[Long, Map[KafkaPartitionKey, Long]]()
-  
-  /* NOT USED - Originally intended for fault-tolerance
- 
-  // In case of a failure, the offets for a particular timestamp will be restored.
-  @transient var restoredOffsets : Map[KafkaPartitionKey, Long] = null
-
- 
-  override protected[streaming] def addMetadata(metadata: Any) {
-    metadata match {
-      case x : KafkaInputDStreamMetadata =>
-        savedOffsets(x.timestamp) = x.data
-        // TOOD: Remove logging
-        logInfo("New saved Offsets: " + savedOffsets)
-      case _ => logInfo("Received unknown metadata: " + metadata.toString)
-    }
-  }
-
-  override protected[streaming] def updateCheckpointData(currentTime: Time) {
-    super.updateCheckpointData(currentTime)
-    if(savedOffsets.size > 0) {
-      // Find the offets that were stored before the checkpoint was initiated
-      val key = savedOffsets.keys.toList.sortWith(_ < _).filter(_ < currentTime.millis).last
-      val latestOffsets = savedOffsets(key)
-      logInfo("Updating KafkaDStream checkpoint data: " + latestOffsets.toString)
-      checkpointData = KafkaDStreamCheckpointData(checkpointData.rdds, latestOffsets)
-      // TODO: This may throw out offsets that are created after the checkpoint,
-      // but it's unlikely we'll need them.
-      savedOffsets.clear()
-    }
-  }
-
-  override protected[streaming] def restoreCheckpointData() {
-    super.restoreCheckpointData()
-    logInfo("Restoring KafkaDStream checkpoint data.")
-    checkpointData match { 
-      case x : KafkaDStreamCheckpointData => 
-        restoredOffsets = x.savedOffsets
-        logInfo("Restored KafkaDStream offsets: " + savedOffsets)
-    }
-  } */
 
-  def createReceiver(): NetworkReceiver[T] = {
-    new KafkaReceiver(host, port,  groupId, topics, initialOffsets, storageLevel)
+  def getReceiver(): NetworkReceiver[T] = {
+    new KafkaReceiver(zkQuorum,  groupId, topics, initialOffsets, storageLevel)
         .asInstanceOf[NetworkReceiver[T]]
   }
 }
 
 private[streaming]
-class KafkaReceiver(host: String, port: Int, groupId: String,
+class KafkaReceiver(zkQuorum: String, groupId: String,
   topics: Map[String, Int], initialOffsets: Map[KafkaPartitionKey, Long], 
   storageLevel: StorageLevel) extends NetworkReceiver[Any] {
 
@@ -111,8 +59,6 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
 
   // Handles pushing data into the BlockManager
   lazy protected val blockGenerator = new BlockGenerator(storageLevel)
-  // Keeps track of the current offsets. Maps from (broker, topic, group, part) -> Offset
-  lazy val offsets = HashMap[KafkaPartitionKey, Long]()
   // Connection to Kafka
   var consumerConnector : ZookeeperConsumerConnector = null
 
@@ -127,24 +73,23 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
     // In case we are using multiple Threads to handle Kafka Messages
     val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _))
 
-    val zooKeeperEndPoint = host + ":" + port
     logInfo("Starting Kafka Consumer Stream with group: " + groupId)
     logInfo("Initial offsets: " + initialOffsets.toString)
-    
+
     // Zookeper connection properties
     val props = new Properties()
-    props.put("zk.connect", zooKeeperEndPoint)
+    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: " + zooKeeperEndPoint)
+    logInfo("Connecting to Zookeper: " + zkQuorum)
     val consumerConfig = new ConsumerConfig(props)
     consumerConnector = Consumer.create(consumerConfig).asInstanceOf[ZookeeperConsumerConnector]
-    logInfo("Connected to " + zooKeeperEndPoint)
+    logInfo("Connected to " + zkQuorum)
 
-    // Reset the Kafka offsets in case we are recovering from a failure
-    resetOffsets(initialOffsets)
+    // 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())
@@ -157,11 +102,11 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
   }
 
   // Overwrites the offets in Zookeper.
-  private def resetOffsets(offsets: Map[KafkaPartitionKey, Long]) {
+  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, 
+      updatePersistentPath(consumerConnector.zkClient,
         topicDirs.consumerOffsetDir + "/" + partitionName, offset.toString)
     }
   }
@@ -172,29 +117,10 @@ class KafkaReceiver(host: String, port: Int, groupId: String,
       logInfo("Starting MessageHandler.")
       stream.takeWhile { msgAndMetadata =>
         blockGenerator += msgAndMetadata.message
-
-        // Updating the offet. The key is (broker, topic, group, partition).
-        val key = KafkaPartitionKey(msgAndMetadata.topicInfo.brokerId, msgAndMetadata.topic, 
-          groupId, msgAndMetadata.topicInfo.partition.partId)
-        val offset = msgAndMetadata.topicInfo.getConsumeOffset
-        offsets.put(key, offset)
-        // logInfo("Handled message: " + (key, offset).toString)
-
         // Keep on handling messages
+
         true
-      }  
+      }
     }
   }
-
-  // NOT USED - Originally intended for fault-tolerance
-  // class KafkaDataHandler(receiver: KafkaReceiver, storageLevel: StorageLevel) 
-  // extends BufferingBlockCreator[Any](receiver, storageLevel) {
-
-  //   override def createBlock(blockId: String, iterator: Iterator[Any]) : Block = {
-  //     // Creates a new Block with Kafka-specific Metadata
-  //     new Block(blockId, iterator, KafkaInputDStreamMetadata(System.currentTimeMillis, offsets.toMap))
-  //   }
-
-  // }
-
 }
diff --git a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
index 8c322dd698099be5426008e0cadf792f002a736c..7385474963ebb70bfd7f8f86a16282e341824c80 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/NetworkInputDStream.scala
@@ -20,7 +20,7 @@ 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 createReceiver() function that creates the receiver object of type
+ * 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
@@ -34,11 +34,11 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming
   val id = ssc.getNewNetworkStreamId()
 
   /**
-   * Creates the receiver object that will be sent to the worker nodes
+   * 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 createReceiver(): NetworkReceiver[T]
+  def getReceiver(): NetworkReceiver[T]
 
   // Nothing to start or stop as both taken care of by the NetworkInputTracker.
   def start() {}
@@ -46,8 +46,15 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming
   def stop() {}
 
   override def compute(validTime: Time): Option[RDD[T]] = {
-    val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)    
-    Some(new BlockRDD[T](ssc.sc, blockIds))
+    // 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]()))
+    }
   }
 }
 
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
index 024bf3bea47832867e2ce419e41e56786fa2c73d..6b310bc0b611c7a6260042e3ddc555a59bed28a2 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/QueueInputDStream.scala
@@ -7,6 +7,7 @@ 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]],
diff --git a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
index 04e6b69b7ba9d41a12a400d417df0150d9bd9c1c..1b2fa567795d4245567a42362a6f45e4f8d96e8b 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/RawInputDStream.scala
@@ -25,7 +25,7 @@ class RawInputDStream[T: ClassManifest](
     storageLevel: StorageLevel
   ) extends NetworkInputDStream[T](ssc_ ) with Logging {
 
-  def createReceiver(): NetworkReceiver[T] = {
+  def getReceiver(): NetworkReceiver[T] = {
     new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]]
   }
 }
diff --git a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
index 263655039cd11719a14e6567cedb2e5cc527d913..343b6915e79a36b02292bace746681474f0e3294 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -3,7 +3,7 @@ package spark.streaming.dstream
 import spark.streaming.StreamingContext._
 
 import spark.RDD
-import spark.rdd.CoGroupedRDD
+import spark.rdd.{CoGroupedRDD, MapPartitionsRDD}
 import spark.Partitioner
 import spark.SparkContext._
 import spark.storage.StorageLevel
@@ -15,7 +15,8 @@ private[streaming]
 class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
     parent: DStream[(K, V)],
     reduceFunc: (V, V) => V,
-    invReduceFunc: (V, V) => V, 
+    invReduceFunc: (V, V) => V,
+    filterFunc: Option[((K, V)) => Boolean],
     _windowDuration: Duration,
     _slideDuration: Duration,
     partitioner: Partitioner
@@ -87,15 +88,18 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
     //
 
     // Get the RDDs of the reduced values in "old time steps"
-    val oldRDDs = reducedStream.slice(previousWindow.beginTime, currentWindow.beginTime - parent.slideDuration)
+    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)
+    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)]()))
+    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
@@ -114,7 +118,9 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
       // 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)
+      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) {
@@ -140,10 +146,12 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
 
     val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues)
 
-    Some(mergedValuesRDD)
+    if (filterFunc.isDefined) {
+      Some(mergedValuesRDD.filter(filterFunc.get))
+    } else {
+      Some(mergedValuesRDD)
+    }
   }
-
-
 }
 
 
diff --git a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
index d42027092b620aa51b4ba36bc3a88310fdbd6299..4af839ad7f03d1e75dd19715a2cb0785a9a91628 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/SocketInputDStream.scala
@@ -15,7 +15,7 @@ class SocketInputDStream[T: ClassManifest](
     storageLevel: StorageLevel
   ) extends NetworkInputDStream[T](ssc_) {
 
-  def createReceiver(): NetworkReceiver[T] = {
+  def getReceiver(): NetworkReceiver[T] = {
     new SocketReceiver(host, port, bytesToObjects, storageLevel)
   }
 }
diff --git a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
index b4506c74aa4e14daf720d560c52ae24b69c5aeac..db62955036e9c1a72a5e11571628a97ca858af53 100644
--- a/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/StateDStream.scala
@@ -48,8 +48,16 @@ class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
             //logDebug("Generating state RDD for time " + validTime)
             return Some(stateRDD)
           }
-          case None => {    // If parent RDD does not exist, then return old state RDD
-            return Some(prevStateRDD)
+          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)
           }
         }
       }
diff --git a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
similarity index 87%
rename from examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala
rename to streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
index 99ed4cdc1c12d13ef8367f2f5323fb7927b51ca2..c69749886289cd6e49401731f7970628e99115c3 100644
--- a/examples/src/main/scala/spark/streaming/examples/twitter/TwitterInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/dstream/TwitterInputDStream.scala
@@ -1,12 +1,11 @@
-package spark.streaming.examples.twitter
+package spark.streaming.dstream
 
 import spark._
 import spark.streaming._
-import dstream.{NetworkReceiver, NetworkInputDStream}
 import storage.StorageLevel
+
 import twitter4j._
 import twitter4j.auth.BasicAuthorization
-import collection.JavaConversions._
 
 /* A stream of Twitter statuses, potentially filtered by one or more keywords.
 *
@@ -14,19 +13,21 @@ import collection.JavaConversions._
 * 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_)  {
+  ) extends NetworkInputDStream[Status](ssc_)  {
 
-  override def createReceiver(): NetworkReceiver[Status] = {
+  override def getReceiver(): NetworkReceiver[Status] = {
     new TwitterReceiver(username, password, filters, storageLevel)
   }
 }
 
+private[streaming]
 class TwitterReceiver(
     username: String,
     password: String,
@@ -50,7 +51,7 @@ class TwitterReceiver(
       def onTrackLimitationNotice(i: Int) {}
       def onScrubGeo(l: Long, l1: Long) {}
       def onStallWarning(stallWarning: StallWarning) {}
-      def onException(e: Exception) {}
+      def onException(e: Exception) { stopOnError(e) }
     })
 
     val query: FilterQuery = new FilterQuery
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/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/RecurringTimer.scala b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
index db715cc295205cfb76ac0b416dfaf039eaaa31f8..8e10276deb9056301f98712637208db3ce2a74b7 100644
--- a/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
+++ b/streaming/src/main/scala/spark/streaming/util/RecurringTimer.scala
@@ -3,9 +3,9 @@ package spark.streaming.util
 private[streaming]
 class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) {
   
-  val minPollTime = 25L
+  private val minPollTime = 25L
   
-  val pollTime = {
+  private val pollTime = {
     if (period / 10.0 > minPollTime) {
       (period / 10.0).toLong
     } else {
@@ -13,11 +13,20 @@ class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) =>
     }  
   }
   
-  val thread = new Thread() {
+  private val thread = new Thread() {
     override def run() { loop }    
   }
   
-  var nextTime = 0L   
+  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
@@ -26,21 +35,14 @@ class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) =>
   }
 
   def start(): Long = {
-    val startTime = (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period
-    start(startTime)
+    start(getStartTime())
   }
 
-  def restart(originalStartTime: Long): Long = {
-    val gap = clock.currentTime - originalStartTime
-    val newStartTime = (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime
-    start(newStartTime)
-  }
-  
-  def stop() { 
+  def stop() {
     thread.interrupt() 
   }
   
-  def loop() {
+  private def loop() {
     try {
       while (true) {
         clock.waitTillTime(nextTime)
diff --git a/streaming/src/test/java/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/spark/streaming/JavaAPISuite.java
index 7b385f609d302bf5cf450f96d4af37bf2f831777..3bed500f73e84548a2f470a414ea68cfbb3df1ac 100644
--- a/streaming/src/test/java/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/spark/streaming/JavaAPISuite.java
@@ -13,6 +13,8 @@ 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;
@@ -22,10 +24,16 @@ 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/.
@@ -34,8 +42,9 @@ public class JavaAPISuite implements Serializable {
 
   @Before
   public void setUp() {
-    ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
-    ssc.checkpoint("checkpoint", new Duration(1000));
+      System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock");
+      ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+    ssc.checkpoint("checkpoint");
   }
 
   @After
@@ -134,29 +143,6 @@ public class JavaAPISuite implements Serializable {
     assertOrderInvariantEquals(expected, result);
   }
 
-  @Test
-  public void testTumble() {
-    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(7,8,9,10,11,12),
-        Arrays.asList(13,14,15,16,17,18));
-
-    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream windowed = stream.tumble(new Duration(2000));
-    JavaTestUtils.attachTestOutputStream(windowed);
-    List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 6, 3);
-
-    assertOrderInvariantEquals(expected, result);
-  }
-
   @Test
   public void testFilter() {
     List<List<String>> inputData = Arrays.asList(
@@ -316,8 +302,9 @@ public class JavaAPISuite implements Serializable {
         Arrays.asList(6,7,8),
         Arrays.asList(9,10,11));
 
-    JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaDStream transformed = stream.transform(new Function<JavaRDD<Integer>, JavaRDD<Integer>>() {
+    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>() {
@@ -719,50 +706,73 @@ public class JavaAPISuite implements Serializable {
   }
 
   @Test
-  public void testCountByKey() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+  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>("california", 2L),
-            new Tuple2<String, Long>("new york", 2L)),
-        Arrays.asList(
-            new Tuple2<String, Long>("california", 2L),
-            new Tuple2<String, Long>("new york", 2L)));
-
-    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
-        ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+      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)));
 
-    JavaPairDStream<String, Long> counted = pairStream.countByKey();
+    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, 2, 2);
+    List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
 
     Assert.assertEquals(expected, result);
   }
 
   @Test
   public void testGroupByKeyAndWindow() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+    List<List<Tuple2<String, Integer>>> inputData = stringIntKVStream;
 
-    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", "dodgers", "giants")),
-            new Tuple2<String, List<String>>("new york", Arrays.asList("rangers", "islanders", "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"))));
+    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, String>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
+    JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+    JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream);
 
-    JavaPairDStream<String, List<String>> groupWindowed =
+    JavaPairDStream<String, List<Integer>> groupWindowed =
         pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000));
     JavaTestUtils.attachTestOutputStream(groupWindowed);
-    List<List<Tuple2<String, List<String>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
+    List<List<Tuple2<String, List<Integer>>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
 
-    Assert.assertEquals(expected, result);
+    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
@@ -847,26 +857,28 @@ public class JavaAPISuite implements Serializable {
   }
 
   @Test
-  public void testCountByKeyAndWindow() {
-    List<List<Tuple2<String, String>>> inputData = stringStringKVStream;
+  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>("california", 2L),
-            new Tuple2<String, Long>("new york", 2L)),
+            new Tuple2<String, Long>("hello", 1L),
+            new Tuple2<String, Long>("world", 1L)),
         Arrays.asList(
-            new Tuple2<String, Long>("california", 4L),
-            new Tuple2<String, Long>("new york", 4L)),
+            new Tuple2<String, Long>("hello", 2L),
+            new Tuple2<String, Long>("world", 1L),
+            new Tuple2<String, Long>("moon", 1L)),
         Arrays.asList(
-            new Tuple2<String, Long>("california", 2L),
-            new Tuple2<String, Long>("new york", 2L)));
+            new Tuple2<String, Long>("hello", 2L),
+            new Tuple2<String, Long>("moon", 1L)));
 
-    JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(
+    JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(
         ssc, inputData, 1);
-    JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream);
-
     JavaPairDStream<String, Long> counted =
-        pairStream.countByKeyAndWindow(new Duration(2000), new Duration(1000));
+      stream.countByValueAndWindow(new Duration(2000), new Duration(1000));
     JavaTestUtils.attachTestOutputStream(counted);
     List<List<Tuple2<String, Long>>> result = JavaTestUtils.runStreams(ssc, 3, 3);
 
@@ -918,6 +930,46 @@ public class JavaAPISuite implements Serializable {
   }
 
   @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;
 
@@ -1091,9 +1143,8 @@ public class JavaAPISuite implements Serializable {
         Arrays.asList(1,4),
         Arrays.asList(8,7));
 
-
     File tempDir = Files.createTempDir();
-    ssc.checkpoint(tempDir.getAbsolutePath(), new Duration(1000));
+    ssc.checkpoint(tempDir.getAbsolutePath());
 
     JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
     JavaDStream letterCount = stream.map(new Function<String, Integer>() {
@@ -1107,14 +1158,16 @@ public class JavaAPISuite implements Serializable {
 
     assertOrderInvariantEquals(expectedInitial, initialResult);
     Thread.sleep(1000);
-
     ssc.stop();
+
     ssc = new JavaStreamingContext(tempDir.getAbsolutePath());
-    ssc.start();
-    List<List<Integer>> finalResult = JavaCheckpointTestUtils.runStreams(ssc, 2, 2);
-    assertOrderInvariantEquals(expectedFinal, finalResult);
+    // 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 {
@@ -1151,19 +1204,19 @@ public class JavaAPISuite implements Serializable {
   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,
+    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 testNetworkTextStream() {
-    JavaDStream test = ssc.networkTextStream("localhost", 12345);
+  public void testSocketTextStream() {
+    JavaDStream test = ssc.socketTextStream("localhost", 12345);
   }
 
   @Test
-  public void testNetworkString() {
+  public void testSocketString() {
     class Converter extends Function<InputStream, Iterable<String>> {
       public Iterable<String> call(InputStream in) {
         BufferedReader reader = new BufferedReader(new InputStreamReader(in));
@@ -1179,7 +1232,7 @@ public class JavaAPISuite implements Serializable {
       }
     }
 
-    JavaDStream test = ssc.networkStream(
+    JavaDStream test = ssc.socketStream(
       "localhost",
       12345,
       new Converter(),
@@ -1192,13 +1245,13 @@ public class JavaAPISuite implements Serializable {
   }
 
   @Test
-  public void testRawNetworkStream() {
-    JavaDStream test = ssc.rawNetworkStream("localhost", 12345);
+  public void testRawSocketStream() {
+    JavaDStream test = ssc.rawSocketStream("localhost", 12345);
   }
 
   @Test
   public void testFlumeStream() {
-    JavaDStream test = ssc.flumeStream("localhost", 12345);
+    JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
   }
 
   @Test
@@ -1206,4 +1259,25 @@ public class JavaAPISuite implements Serializable {
     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
index 56349837e5e6ba5895c2de50103a24e9d9e94bfe..64a7e7cbf9a367285b9a3704f781235dac099bbb 100644
--- a/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/spark/streaming/JavaTestUtils.scala
@@ -31,8 +31,9 @@ trait JavaTestBase extends TestSuiteBase {
    * 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]](
-    dstream: JavaDStreamLike[T, This]) = {
+  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,
@@ -57,6 +58,7 @@ trait JavaTestBase extends TestSuiteBase {
 }
 
 object JavaTestUtils extends JavaTestBase {
+  override def maxWaitTimeMillis = 20000
 
 }
 
diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties
index edfa1243fa2bc54ca72d5af3b4d0ad7df9263a56..59c445e63f7974163e77d47257644474dd14e62f 100644
--- a/streaming/src/test/resources/log4j.properties
+++ b/streaming/src/test/resources/log4j.properties
@@ -1,5 +1,6 @@
 # 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
diff --git a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
index 4a036f071074f8ed4228e61395e12e585be17d07..8fce91853c77eed0f5d3e5e48aa2e93f13988e3e 100644
--- a/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/BasicOperationsSuite.scala
@@ -6,6 +6,8 @@ import util.ManualClock
 
 class BasicOperationsSuite extends TestSuiteBase {
 
+  System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
+
   override def framework() = "BasicOperationsSuite"
 
   after {
@@ -22,7 +24,7 @@ class BasicOperationsSuite extends TestSuiteBase {
     )
   }
 
-  test("flatmap") {
+  test("flatMap") {
     val input = Seq(1 to 4, 5 to 8, 9 to 12)
     testOperation(
       input,
@@ -86,6 +88,23 @@ class BasicOperationsSuite extends TestSuiteBase {
     )
   }
 
+  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() ),
@@ -165,6 +184,71 @@ class BasicOperationsSuite extends TestSuiteBase {
     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")
 
diff --git a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
index 563a7d14587b591aadb5d7587348607b28d0bf46..cac86deeaf3492cb298c26d9812cc82e71fd83cf 100644
--- a/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/CheckpointSuite.scala
@@ -1,5 +1,6 @@
 package spark.streaming
 
+import dstream.FileInputDStream
 import spark.streaming.StreamingContext._
 import java.io.File
 import runtime.RichInt
@@ -7,9 +8,19 @@ 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))
   }
@@ -28,21 +39,18 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
   override def batchDuration = Milliseconds(500)
 
-  override def checkpointInterval = batchDuration
-
   override def actuallyWait = true
 
-  test("basic stream+rdd recovery") {
+  test("basic rdd checkpoints + dstream graph checkpoint recovery") {
 
     assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
-    assert(checkpointInterval === batchDuration, "checkpointInterval for this test much be same as batchDuration")
 
     System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
 
     val stateStreamCheckpointInterval = Seconds(1)
 
     // this ensure checkpointing occurs at least once
-    val firstNumBatches = (stateStreamCheckpointInterval / batchDuration) * 2
+    val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2
     val secondNumBatches = firstNumBatches
 
     // Setup the streams
@@ -62,10 +70,10 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     // 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()
-    runStreamsWithRealDelay(ssc, firstNumBatches)
-    logInfo("Checkpoint data of state stream = \n[" + stateStream.checkpointData.rdds.mkString(",\n") + "]")
-    assert(!stateStream.checkpointData.rdds.isEmpty, "No checkpointed RDDs in state stream before first failure")
-    stateStream.checkpointData.rdds.foreach {
+    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")
@@ -74,8 +82,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
 
     // 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.rdds.map(x => new File(x._2.toString))
-    runStreamsWithRealDelay(ssc, secondNumBatches)
+    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()
 
@@ -90,9 +98,9 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     // Run one batch to generate a new checkpoint file and check whether some RDD
     // is present in the checkpoint data or not
     ssc.start()
-    runStreamsWithRealDelay(ssc, 1)
-    assert(!stateStream.checkpointData.rdds.isEmpty, "No checkpointed RDDs in state stream before second failure")
-    stateStream.checkpointData.rdds.foreach {
+    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(),
@@ -111,13 +119,16 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     // 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()
-    runStreamsWithRealDelay(ssc, 4)
+    advanceTimeWithRealDelay(ssc, 4)
     ssc.stop()
     System.clearProperty("spark.streaming.manualClock.jump")
     ssc = null
   }
 
-  test("map and reduceByKey") {
+  // 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(_ + _),
@@ -126,7 +137,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     )
   }
 
-  test("reduceByKeyAndWindowInv") {
+
+  // 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
@@ -139,7 +154,11 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     testCheckpointedOperation(input, operation, output, 7)
   }
 
-  test("updateStateByKey") {
+
+  // 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]) => {
@@ -154,11 +173,126 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     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 restart the operation
+   * 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]],
@@ -172,11 +306,14 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     val totalNumBatches = input.size
     val nextNumBatches = totalNumBatches - initialNumBatches
     val initialNumExpectedOutputs = initialNumBatches
-    val nextNumExpectedOutputs = expectedOutput.size - initialNumExpectedOutputs
+    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)
-    val output = runStreams[V](ssc, initialNumBatches, initialNumExpectedOutputs)
+    ssc.start()
+    val output = advanceTimeWithRealDelay[V](ssc, initialNumBatches)
+    ssc.stop()
     verifyOutput[V](output, expectedOutput.take(initialNumBatches), true)
     Thread.sleep(1000)
 
@@ -187,16 +324,20 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
       "\n-------------------------------------------\n"
     )
     ssc = new StreamingContext(checkpointDir)
-    val outputNew = runStreams[V](ssc, nextNumBatches, nextNumExpectedOutputs)
+    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 wait for the expected amount of time for each batch.
+   * It also waits for the expected amount of time for each batch.
    */
-  def runStreamsWithRealDelay(ssc: StreamingContext, numBatches: Long) {
+  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) {
@@ -205,6 +346,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
     }
     logInfo("Manual clock after advancing = " + clock.time)
     Thread.sleep(batchDuration.milliseconds)
-  }
 
+    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
+    outputStream.output
+  }
 }
\ 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
index c4cfffbfc1ef3cf8f8c79a4963525a0ee936f28f..a5fa7ab92dd6bf9564e1f33aba39d82a51deda04 100644
--- a/streaming/src/test/scala/spark/streaming/FailureSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/FailureSuite.scala
@@ -1,191 +1,40 @@
 package spark.streaming
 
-import org.scalatest.BeforeAndAfter
-import org.apache.commons.io.FileUtils
+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 scala.runtime.RichInt
-import scala.util.Random
-import spark.streaming.StreamingContext._
+import org.apache.commons.io.FileUtils
 import collection.mutable.ArrayBuffer
-import spark.Logging
+
 
 /**
  * This testsuite tests master failures at random times while the stream is running using
  * the real clock.
  */
-class FailureSuite extends TestSuiteBase with BeforeAndAfter {
+class FailureSuite extends FunSuite with BeforeAndAfter with Logging {
+
+  var directory = "FailureSuite"
+  val numBatches = 30
+  val batchDuration = Milliseconds(1000)
 
   before {
-    FileUtils.deleteDirectory(new File(checkpointDir))
+    FileUtils.deleteDirectory(new File(directory))
   }
 
   after {
-    FailureSuite.reset()
-    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")
-  }
-
-  override def framework = "CheckpointSuite"
-
-  override def batchDuration = Milliseconds(500)
-
-  override def checkpointDir = "checkpoint"
-
-  override def checkpointInterval = batchDuration
-
-  test("multiple failures with updateStateByKey") {
-    val n = 30
-    // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ...
-    val input = (1 to n).map(i => (1 to i).map(_ =>"a").toSeq).toSeq
-    // Last output: [ (a, 465) ]   for n=30
-    val lastOutput = Seq( ("a", (1 to n).reduce(_ + _)) )
-
-    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(Seconds(2))
-     .map(t => (t._1, t._2.self))
-    }
-
-    testOperationWithMultipleFailures(input, operation, lastOutput, n, n)
-  }
-
-  test("multiple failures with reduceByKeyAndWindow") {
-    val n = 30
-    val w = 100
-    assert(w > n, "Window should be much larger than the number of input sets in this test")
-    // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ...
-    val input = (1 to n).map(i => (1 to i).map(_ =>"a").toSeq).toSeq
-    // Last output: [ (a, 465) ]
-    val lastOutput = Seq( ("a", (1 to n).reduce(_ + _)) )
-
-    val operation = (st: DStream[String]) => {
-      st.map(x => (x, 1))
-        .reduceByKeyAndWindow(_ + _, _ - _, batchDuration * w, batchDuration)
-        .checkpoint(Seconds(2))
-    }
-
-    testOperationWithMultipleFailures(input, operation, lastOutput, n, n)
-  }
-
-
-  /**
-   * Tests stream operation with multiple master failures, and verifies whether the
-   * final set of output values is as expected or not. Checking the final value is
-   * proof that no intermediate data was lost due to master failures.
-   */
-  def testOperationWithMultipleFailures[U: ClassManifest, V: ClassManifest](
-    input: Seq[Seq[U]],
-    operation: DStream[U] => DStream[V],
-    lastExpectedOutput: Seq[V],
-    numBatches: Int,
-    numExpectedOutput: Int
-  ) {
-    var ssc = setupStreams[U, V](input, operation)
-    val mergedOutput = new ArrayBuffer[Seq[V]]()
-
-    var totalTimeRan = 0L
-    while(totalTimeRan <= numBatches * batchDuration.milliseconds * 2) {
-      new KillingThread(ssc, numBatches * batchDuration.milliseconds.toInt / 4).start()
-      val (output, timeRan) = runStreamsWithRealClock[V](ssc, numBatches, numExpectedOutput)
-
-      mergedOutput ++= output
-      totalTimeRan += timeRan
-      logInfo("New output = " + output)
-      logInfo("Merged output = " + mergedOutput)
-      logInfo("Total time spent = " + totalTimeRan)
-      val sleepTime = Random.nextInt(numBatches * batchDuration.milliseconds.toInt / 8)
-      logInfo(
-        "\n-------------------------------------------\n" +
-        "   Restarting stream computation in " + sleepTime + " ms   " +
-        "\n-------------------------------------------\n"
-      )
-      Thread.sleep(sleepTime)
-      FailureSuite.failed = false
-      ssc = new StreamingContext(checkpointDir)
-    }
-    ssc.stop()
-    ssc = null
-
-    // Verify whether the last output is the expected one
-    val lastOutput = mergedOutput(mergedOutput.lastIndexWhere(!_.isEmpty))
-    assert(lastOutput.toSet === lastExpectedOutput.toSet)
-    logInfo("Finished computation after " + FailureSuite.failureCount + " failures")
+    FileUtils.deleteDirectory(new File(directory))
   }
 
-  /**
-   * Runs the streams set up in `ssc` on real clock until the expected max number of
-   */
-  def runStreamsWithRealClock[V: ClassManifest](
-    ssc: StreamingContext,
-    numBatches: Int,
-    maxExpectedOutput: Int
-  ): (Seq[Seq[V]], Long) = {
-
-    System.clearProperty("spark.streaming.clock")
-
-    assert(numBatches > 0, "Number of batches to run stream computation is zero")
-    assert(maxExpectedOutput > 0, "Max expected outputs after " + numBatches + " is zero")
-    logInfo("numBatches = " + numBatches + ", maxExpectedOutput = " + maxExpectedOutput)
-
-    // Get the output buffer
-    val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
-    val output = outputStream.output
-    val waitTime = (batchDuration.milliseconds * (numBatches.toDouble + 0.5)).toLong
-    val startTime = System.currentTimeMillis()
-
-    try {
-      // Start computation
-      ssc.start()
-
-      // Wait until expected number of output items have been generated
-      while (output.size < maxExpectedOutput && System.currentTimeMillis() - startTime < waitTime && !FailureSuite.failed) {
-        logInfo("output.size = " + output.size + ", maxExpectedOutput = " + maxExpectedOutput)
-        Thread.sleep(100)
-      }
-    } catch {
-      case e: Exception => logInfo("Exception while running streams: " + e)
-    } finally {
-      ssc.stop()
-    }
-    val timeTaken = System.currentTimeMillis() - startTime
-    logInfo("" + output.size + " sets of output generated in " + timeTaken + " ms")
-    (output, timeTaken)
+  test("multiple failures with map") {
+    MasterFailureTest.testMap(directory, numBatches, batchDuration)
   }
 
-
-}
-
-object FailureSuite {
-  var failed = false
-  var failureCount = 0
-
-  def reset() {
-    failed = false
-    failureCount = 0
+  test("multiple failures with updateStateByKey") {
+    MasterFailureTest.testUpdateStateByKey(directory, numBatches, batchDuration)
   }
 }
 
-class KillingThread(ssc: StreamingContext, maxKillWaitTime: Int) extends Thread with Logging {
-  initLogging()
-
-  override def run() {
-    var minKillWaitTime = if (FailureSuite.failureCount == 0) 3000 else 1000 // to allow the first checkpoint
-    val killWaitTime = minKillWaitTime + Random.nextInt(maxKillWaitTime)
-    logInfo("Kill wait time = " + killWaitTime)
-    Thread.sleep(killWaitTime.toLong)
-    logInfo(
-      "\n---------------------------------------\n" +
-      "Killing streaming context after " + killWaitTime + " ms" +
-      "\n---------------------------------------\n"
-    )
-    if (ssc != null) ssc.stop()
-    FailureSuite.failed = true
-    FailureSuite.failureCount += 1
-  }
-}
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
index 70ae6e3934cfd3403e47903a3adaa46d024fddcb..4d33857b25ed5c1ecf4b3fdf783db5cc61214c89 100644
--- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -1,5 +1,11 @@
 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}
@@ -7,6 +13,7 @@ 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
@@ -19,40 +26,30 @@ 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 {
-    
+
   System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
 
   val testPort = 9999
-  var testServer: TestServer = null
-  var testDir: File = null
 
   override def checkpointDir = "checkpoint"
 
   after {
-    FileUtils.deleteDirectory(new File(checkpointDir))
-    if (testServer != null) {
-      testServer.stop()
-      testServer = null
-    }
-    if (testDir != null && testDir.exists()) {
-      FileUtils.deleteDirectory(testDir)
-      testDir = null
-    }
-
     // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
     System.clearProperty("spark.driver.port")
   }
 
-  test("network input stream") {
+
+  test("socket input stream") {
     // Start the server
-    testServer = new TestServer(testPort)
+    val testServer = new TestServer()
     testServer.start()
 
     // Set up the streaming context and input streams
     val ssc = new StreamingContext(master, framework, batchDuration)
-    val networkStream = ssc.networkTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+    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)
@@ -93,51 +90,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     }
   }
 
-  test("network input stream with checkpoint") {
-    // Start the server
-    testServer = new TestServer(testPort)
-    testServer.start()
-
-    // Set up the streaming context and input streams
-    var ssc = new StreamingContext(master, framework, batchDuration)
-    ssc.checkpoint(checkpointDir, checkpointInterval)
-    val networkStream = ssc.networkTextStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
-    var outputStream = new TestOutputStream(networkStream, new ArrayBuffer[Seq[String]])
-    ssc.registerOutputStream(outputStream)
-    ssc.start()
-
-    // Feed data to the server to send to the network receiver
-    var clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
-    for (i <- Seq(1, 2, 3)) {
-      testServer.send(i.toString + "\n")
-      Thread.sleep(100)
-      clock.addToTime(batchDuration.milliseconds)
-    }
-    Thread.sleep(500)
-    assert(outputStream.output.size > 0)
-    ssc.stop()
-
-    // Restart stream computation from checkpoint and feed more data to see whether
-    // they are being received and processed
-    logInfo("*********** RESTARTING ************")
-    ssc = new StreamingContext(checkpointDir)
-    ssc.start()
-    clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
-    for (i <- Seq(4, 5, 6)) {
-      testServer.send(i.toString + "\n")
-      Thread.sleep(100)
-      clock.addToTime(batchDuration.milliseconds)
-    }
-    Thread.sleep(500)
-    outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[String]]
-    assert(outputStream.output.size > 0)
-    ssc.stop()
-  }
 
   test("flume input stream") {
     // Set up the streaming context and input streams
     val ssc = new StreamingContext(master, framework, batchDuration)
-    val flumeStream = ssc.flumeStream("localhost", 33333, StorageLevel.MEMORY_AND_DISK)
+    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)
@@ -146,8 +103,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
 
     val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
     val input = Seq(1, 2, 3, 4, 5)
-
-    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", 33333));
+    Thread.sleep(1000)
+    val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort));
     val client = SpecificRequestor.getClient(
       classOf[AvroSourceProtocol], transceiver);
 
@@ -182,42 +139,33 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
     }
   }
 
-  test("file input stream") {
 
-    // Create a temporary directory
-    testDir = {
-      var temp = File.createTempFile(".temp.", Random.nextInt().toString)
-      temp.delete()
-      temp.mkdirs()
-      logInfo("Created temp dir " + temp)
-      temp
-    }
+  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 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)
+    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 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) {
-      FileUtils.writeStringToFile(new File(testDir, i.toString), input(i).toString + "\n")
-      Thread.sleep(500)
-      clock.addToTime(batchDuration.milliseconds)
-      //Thread.sleep(100)
+      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()
-    /*while (output.size < expectedOutput.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
-      logInfo("output.size = " + output.size + ", expectedOutput.size = " + expectedOutput.size)
-      Thread.sleep(100)
-    }*/
     Thread.sleep(1000)
     val timeTaken = System.currentTimeMillis() - startTime
     assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
@@ -226,80 +174,81 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
 
     // Verify whether data received by Spark Streaming was as expected
     logInfo("--------------------------------")
-    logInfo("output.size = " + outputBuffer.size)
-    logInfo("output")
+    logInfo("output, size = " + outputBuffer.size)
     outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]"))
-    logInfo("expected output.size = " + expectedOutput.size)
-    logInfo("expected output")
+    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.size === expectedOutput.size)
-    for (i <- 0 until output.size) {
-      assert(output(i).size === 1)
-      assert(output(i).head.toString === expectedOutput(i))
-    }
+    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("file input stream with checkpoint") {
-    // Create a temporary directory
-    testDir = {
-      var temp = File.createTempFile(".temp.", Random.nextInt().toString)
-      temp.delete()
-      temp.mkdirs()
-      logInfo("Created temp dir " + temp)
-      temp
-    }
+
+  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
-    var ssc = new StreamingContext(master, framework, batchDuration)
-    ssc.checkpoint(checkpointDir, checkpointInterval)
-    val filestream = ssc.textFileStream(testDir.toString)
-    var outputStream = new TestOutputStream(filestream, new ArrayBuffer[Seq[String]])
+    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()
 
-    // Create files and advance manual clock to process them
-    var clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+    // 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 <- Seq(1, 2, 3)) {
-      FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
-      Thread.sleep(100)
+    for (i <- 0 until input.size) {
+      testServer.send(input(i).toString)
+      Thread.sleep(500)
       clock.addToTime(batchDuration.milliseconds)
     }
-    Thread.sleep(500)
-    logInfo("Output = " + outputStream.output.mkString(","))
-    assert(outputStream.output.size > 0)
+    Thread.sleep(1000)
+    logInfo("Stopping server")
+    testServer.stop()
+    logInfo("Stopping context")
     ssc.stop()
 
-    // Restart stream computation from checkpoint and create more files to see whether
-    // they are being processed
-    logInfo("*********** RESTARTING ************")
-    ssc = new StreamingContext(checkpointDir)
-    ssc.start()
-    clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
-    Thread.sleep(500)
-    for (i <- Seq(4, 5, 6)) {
-      FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n")
-      Thread.sleep(100)
-      clock.addToTime(batchDuration.milliseconds)
+    // 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))
     }
-    Thread.sleep(500)
-    outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[String]]
-    logInfo("Output = " + outputStream.output.mkString(","))
-    assert(outputStream.output.size > 0)
-    ssc.stop()
   }
 }
 
 
-class TestServer(port: Int) extends Logging {
+/** This is server to test the network input stream */
+class TestServer() extends Logging {
 
   val queue = new ArrayBlockingQueue[String](100)
 
-  val serverSocket = new ServerSocket(port)
+  val serverSocket = new ServerSocket(0)
 
   val servingThread = new Thread() {
     override def run() {
@@ -341,11 +290,13 @@ class TestServer(port: Int) extends Logging {
   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(9999)
+    val s = new TestServer()
     s.start()
     while(true) {
       Thread.sleep(1000)
@@ -353,3 +304,15 @@ object TestServer {
     }
   }
 }
+
+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
index 49129f39640c4d906dee6604fb0b7e1d84e76fee..ad6aa79d102339bc0595d30463d47bdf6ee183fd 100644
--- a/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/spark/streaming/TestSuiteBase.scala
@@ -28,6 +28,11 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[
     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)
@@ -58,20 +63,25 @@ class TestOutputStream[T: ClassManifest](parent: DStream[T], val output: ArrayBu
  */
 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"
 
-  def checkpointInterval = batchDuration
-
+  // 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
 
   /**
@@ -86,7 +96,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
     // Create StreamingContext
     val ssc = new StreamingContext(master, framework, batchDuration)
     if (checkpointDir != null) {
-      ssc.checkpoint(checkpointDir, checkpointInterval)
+      ssc.checkpoint(checkpointDir)
     }
 
     // Setup the stream computation
@@ -111,7 +121,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
     // Create StreamingContext
     val ssc = new StreamingContext(master, framework, batchDuration)
     if (checkpointDir != null) {
-      ssc.checkpoint(checkpointDir, checkpointInterval)
+      ssc.checkpoint(checkpointDir)
     }
 
     // Setup the stream computation
@@ -135,9 +145,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
       numBatches: Int,
       numExpectedOutput: Int
     ): Seq[Seq[V]] = {
-
-    System.setProperty("spark.streaming.clock", "spark.streaming.util.ManualClock")
-
     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)
@@ -181,7 +188,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
     } finally {
       ssc.stop()
     }
-
     output
   }
 
diff --git a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
index cd9608df530af437c42befac4589ad7784ed09a0..1b66f3bda20ad0f112295e529f4a3f1419c167ed 100644
--- a/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/WindowOperationsSuite.scala
@@ -5,6 +5,8 @@ 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
@@ -82,12 +84,9 @@ class WindowOperationsSuite extends TestSuiteBase {
   )
 
   /*
-  The output of the reduceByKeyAndWindow with inverse reduce function is
-  different from the naive reduceByKeyAndWindow. Even if the count of a
-  particular key is 0, the key does not get eliminated from the RDDs of
-  ReducedWindowedDStream. This causes the number of keys in these RDDs to
-  increase forever. A more generalized version that allows elimination of
-  keys should be considered.
+  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(
@@ -175,31 +174,31 @@ class WindowOperationsSuite extends TestSuiteBase {
 
   // Testing reduceByKeyAndWindow (with invertible reduce function)
 
-  testReduceByKeyAndWindowInv(
+  testReduceByKeyAndWindowWithInverse(
     "basic reduction",
     Seq(Seq(("a", 1), ("a", 3)) ),
     Seq(Seq(("a", 4)) )
   )
 
-  testReduceByKeyAndWindowInv(
+  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)) )
   )
 
-  testReduceByKeyAndWindowInv(
+  testReduceByKeyAndWindowWithInverse(
     "new key added into window",
     Seq( Seq(("a", 1)), Seq(("a", 1), ("b", 1)) ),
     Seq( Seq(("a", 1)), Seq(("a", 2), ("b", 1)) )
   )
 
-  testReduceByKeyAndWindowInv(
+  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)) )
   )
 
-  testReduceByKeyAndWindowInv(
+  testReduceByKeyAndWindowWithInverse(
     "larger slide time",
     largerSlideInput,
     largerSlideReduceOutput,
@@ -207,7 +206,9 @@ class WindowOperationsSuite extends TestSuiteBase {
     Seconds(2)
   )
 
-  testReduceByKeyAndWindowInv("big test", bigInput, bigReduceInvOutput)
+  testReduceByKeyAndWindowWithInverse("big test", bigInput, bigReduceInvOutput)
+
+  testReduceByKeyAndWindowWithFilteredInverse("big test", bigInput, bigReduceOutput)
 
   test("groupByKeyAndWindow") {
     val input = bigInput
@@ -235,14 +236,14 @@ class WindowOperationsSuite extends TestSuiteBase {
     testOperation(input, operation, expectedOutput, numBatches, true)
   }
 
-  test("countByKeyAndWindow") {
-    val input = Seq(Seq(("a", 1)), Seq(("b", 1), ("b", 2)), Seq(("a", 10), ("b", 20)))
+  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, Int)]) => {
-      s.countByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt))
+    val operation = (s: DStream[String]) => {
+      s.countByValueAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toInt))
     }
     testOperation(input, operation, expectedOutput, numBatches, true)
   }
@@ -272,29 +273,50 @@ class WindowOperationsSuite extends TestSuiteBase {
     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(_ + _, windowDuration, slideDuration).persist()
+        s.reduceByKeyAndWindow((x: Int, y: Int) => x + y, windowDuration, slideDuration)
       }
       testOperation(input, operation, expectedOutput, numBatches, true)
     }
   }
 
-  def testReduceByKeyAndWindowInv(
+  def testReduceByKeyAndWindowWithInverse(
     name: String,
     input: Seq[Seq[(String, Int)]],
     expectedOutput: Seq[Seq[(String, Int)]],
     windowDuration: Duration = Seconds(2),
     slideDuration: Duration = Seconds(1)
   ) {
-    test("reduceByKeyAndWindowInv - " + name) {
+    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)
-         .persist()
          .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)
+    }
+  }
 }